From 8e13d1f1eccf7881e58cbd3aa771510fd52b3751 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 28 Oct 2022 16:41:10 +0000 Subject: [PATCH] Improve and refactor Kafka/StorageMQ/NATS and data formats --- .../table-engines/integrations/kafka.md | 38 +- .../table-engines/integrations/nats.md | 24 +- .../table-engines/integrations/rabbitmq.md | 67 +++- docs/en/operations/settings/settings.md | 6 + .../functions/type-conversion-functions.md | 70 +++- .../functions/type-conversion-functions.md | 70 +++- src/Formats/FormatFactory.cpp | 16 +- src/Formats/FormatFactory.h | 10 +- src/Formats/ProtobufReader.cpp | 18 +- src/Formats/ProtobufReader.h | 8 +- src/Formats/ProtobufSerializer.cpp | 5 + src/Formats/ProtobufSerializer.h | 1 + src/Functions/formatRow.cpp | 43 +- src/IO/WriteBufferFromVector.h | 11 +- src/Interpreters/executeQuery.cpp | 1 - src/Processors/Formats/IOutputFormat.cpp | 1 + src/Processors/Formats/IOutputFormat.h | 30 +- src/Processors/Formats/IRowOutputFormat.cpp | 7 +- src/Processors/Formats/IRowOutputFormat.h | 30 +- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 10 +- .../Formats/Impl/ArrowBlockOutputFormat.h | 1 + .../Formats/Impl/AvroRowInputFormat.cpp | 8 +- .../Formats/Impl/AvroRowInputFormat.h | 1 + .../Formats/Impl/AvroRowOutputFormat.cpp | 61 +-- .../Formats/Impl/AvroRowOutputFormat.h | 10 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 7 +- .../Formats/Impl/BinaryRowOutputFormat.h | 2 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 7 +- .../Formats/Impl/CSVRowOutputFormat.h | 2 +- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 6 +- .../Formats/Impl/CapnProtoRowOutputFormat.h | 1 - .../Impl/CustomSeparatedRowOutputFormat.cpp | 7 +- .../Impl/CustomSeparatedRowOutputFormat.h | 2 +- .../Formats/Impl/HiveTextRowInputFormat.cpp | 19 +- .../Formats/Impl/HiveTextRowInputFormat.h | 10 +- .../Impl/JSONColumnsBlockOutputFormat.cpp | 1 - .../Impl/JSONColumnsBlockOutputFormatBase.cpp | 1 - ...ONColumnsWithMetadataBlockOutputFormat.cpp | 8 +- ...JSONColumnsWithMetadataBlockOutputFormat.h | 3 +- .../JSONCompactColumnsBlockOutputFormat.cpp | 1 - .../Impl/JSONCompactEachRowRowInputFormat.cpp | 1 - .../JSONCompactEachRowRowOutputFormat.cpp | 6 +- .../Impl/JSONCompactEachRowRowOutputFormat.h | 1 - .../Impl/JSONCompactRowOutputFormat.cpp | 9 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 1 - .../Impl/JSONEachRowRowInputFormat.cpp | 1 + .../Impl/JSONEachRowRowOutputFormat.cpp | 7 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 1 - ...JSONEachRowWithProgressRowOutputFormat.cpp | 8 +- .../Impl/JSONObjectEachRowRowOutputFormat.cpp | 7 +- .../Impl/JSONObjectEachRowRowOutputFormat.h | 1 - .../Formats/Impl/JSONRowOutputFormat.cpp | 16 +- .../Formats/Impl/JSONRowOutputFormat.h | 4 +- .../Formats/Impl/MarkdownRowOutputFormat.cpp | 7 +- .../Formats/Impl/MarkdownRowOutputFormat.h | 2 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 7 +- .../Formats/Impl/MsgPackRowOutputFormat.h | 2 +- .../Formats/Impl/MySQLOutputFormat.cpp | 1 - src/Processors/Formats/Impl/NativeFormat.cpp | 1 - src/Processors/Formats/Impl/NullFormat.cpp | 1 - .../Impl/ODBCDriver2BlockOutputFormat.cpp | 2 +- .../Formats/Impl/ORCBlockOutputFormat.cpp | 6 +- .../Formats/Impl/ORCBlockOutputFormat.h | 1 + .../Impl/ParallelFormattingOutputFormat.h | 11 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 6 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 1 + .../Formats/Impl/PostgreSQLOutputFormat.cpp | 1 - .../Formats/Impl/PrettyBlockOutputFormat.h | 6 +- .../Impl/PrometheusTextOutputFormat.cpp | 6 +- .../Formats/Impl/PrometheusTextOutputFormat.h | 1 - .../Formats/Impl/ProtobufListInputFormat.cpp | 6 + .../Formats/Impl/ProtobufListInputFormat.h | 2 + .../Formats/Impl/ProtobufListOutputFormat.cpp | 12 +- .../Formats/Impl/ProtobufListOutputFormat.h | 2 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 6 + .../Formats/Impl/ProtobufRowInputFormat.h | 2 + .../Formats/Impl/ProtobufRowOutputFormat.cpp | 10 +- .../Formats/Impl/ProtobufRowOutputFormat.h | 1 - .../Formats/Impl/RawBLOBRowOutputFormat.cpp | 8 +- .../Formats/Impl/RawBLOBRowOutputFormat.h | 3 +- .../Formats/Impl/SQLInsertRowOutputFormat.cpp | 11 +- .../Formats/Impl/SQLInsertRowOutputFormat.h | 12 +- .../Formats/Impl/TSKVRowOutputFormat.cpp | 7 +- .../Formats/Impl/TSKVRowOutputFormat.h | 2 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 6 +- .../Impl/TabSeparatedRowOutputFormat.h | 1 - .../Impl/TemplateBlockOutputFormat.cpp | 11 +- .../Formats/Impl/TemplateBlockOutputFormat.h | 1 + .../Formats/Impl/TemplateRowInputFormat.cpp | 1 + .../Formats/Impl/ValuesRowOutputFormat.cpp | 7 +- .../Formats/Impl/ValuesRowOutputFormat.h | 2 +- .../Formats/Impl/VerticalRowOutputFormat.cpp | 7 +- .../Formats/Impl/VerticalRowOutputFormat.h | 7 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 14 +- .../Formats/Impl/XMLRowOutputFormat.h | 5 +- .../OutputFormatWithUTF8ValidationAdaptor.h | 14 +- .../examples/comma_separated_streams.cpp | 3 +- src/Storages/IMessageProducer.cpp | 37 ++ src/Storages/IMessageProducer.h | 67 ++++ src/Storages/Kafka/Buffer_fwd.h | 14 - src/Storages/Kafka/KafkaBlockSink.cpp | 52 --- ...romKafkaConsumer.cpp => KafkaConsumer.cpp} | 92 ++--- ...ferFromKafkaConsumer.h => KafkaConsumer.h} | 19 +- src/Storages/Kafka/KafkaProducer.cpp | 139 +++++++ ...ufferToKafkaProducer.h => KafkaProducer.h} | 41 +- src/Storages/Kafka/KafkaSettings.h | 24 +- src/Storages/Kafka/KafkaSink.h | 35 -- src/Storages/Kafka/KafkaSource.cpp | 67 ++-- src/Storages/Kafka/KafkaSource.h | 6 +- src/Storages/Kafka/StorageKafka.cpp | 85 ++-- src/Storages/Kafka/StorageKafka.h | 21 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 175 --------- src/Storages/MeiliSearch/SinkMeiliSearch.cpp | 2 +- src/Storages/MessageQueueSink.cpp | 78 ++++ src/Storages/MessageQueueSink.h | 66 ++++ src/Storages/NATS/Buffer_fwd.h | 14 - src/Storages/NATS/NATSConnection.h | 1 - ...rFromNATSConsumer.cpp => NATSConsumer.cpp} | 43 +- ...ufferFromNATSConsumer.h => NATSConsumer.h} | 14 +- src/Storages/NATS/NATSHandler.cpp | 2 +- src/Storages/NATS/NATSProducer.cpp | 126 ++++++ src/Storages/NATS/NATSProducer.h | 57 +++ src/Storages/NATS/NATSSettings.h | 1 + src/Storages/NATS/NATSSink.cpp | 56 --- src/Storages/NATS/NATSSink.h | 31 -- src/Storages/NATS/NATSSource.cpp | 31 +- src/Storages/NATS/NATSSource.h | 8 +- src/Storages/NATS/StorageNATS.cpp | 77 ++-- src/Storages/NATS/StorageNATS.h | 24 +- .../NATS/WriteBufferToNATSProducer.cpp | 183 --------- src/Storages/NATS/WriteBufferToNATSProducer.h | 81 ---- src/Storages/RabbitMQ/Buffer_fwd.h | 14 - ...bitMQConsumer.cpp => RabbitMQConsumer.cpp} | 51 +-- ...mRabbitMQConsumer.h => RabbitMQConsumer.h} | 16 +- ...bitMQProducer.cpp => RabbitMQProducer.cpp} | 154 +++----- ...oRabbitMQProducer.h => RabbitMQProducer.h} | 71 ++-- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + src/Storages/RabbitMQ/RabbitMQSink.cpp | 56 --- src/Storages/RabbitMQ/RabbitMQSink.h | 32 -- src/Storages/RabbitMQ/RabbitMQSource.cpp | 54 +-- src/Storages/RabbitMQ/RabbitMQSource.h | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 96 ++--- src/Storages/RabbitMQ/StorageRabbitMQ.h | 31 +- src/Storages/StorageFile.cpp | 3 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- tests/fuzz/all.dict | 1 + tests/fuzz/dictionaries/functions.dict | 1 + tests/integration/test_storage_kafka/test.py | 319 ++++++++++++++- tests/integration/test_storage_nats/test.py | 370 ++++++++++++++++++ .../integration/test_storage_rabbitmq/test.py | 352 +++++++++++++++++ .../0_stateless/01420_format_row.reference | 117 ++++-- .../0_stateless/01420_format_row.sql.j2 | 11 +- .../0_stateless/02113_format_row.reference | 40 +- .../queries/0_stateless/02113_format_row.sql | 1 + ...new_functions_must_be_documented.reference | 1 + tests/queries/0_stateless/helpers/client.py | 2 +- 157 files changed, 2770 insertions(+), 1717 deletions(-) create mode 100644 src/Storages/IMessageProducer.cpp create mode 100644 src/Storages/IMessageProducer.h delete mode 100644 src/Storages/Kafka/Buffer_fwd.h delete mode 100644 src/Storages/Kafka/KafkaBlockSink.cpp rename src/Storages/Kafka/{ReadBufferFromKafkaConsumer.cpp => KafkaConsumer.cpp} (89%) rename src/Storages/Kafka/{ReadBufferFromKafkaConsumer.h => KafkaConsumer.h} (86%) create mode 100644 src/Storages/Kafka/KafkaProducer.cpp rename src/Storages/Kafka/{WriteBufferToKafkaProducer.h => KafkaProducer.h} (51%) delete mode 100644 src/Storages/Kafka/KafkaSink.h delete mode 100644 src/Storages/Kafka/WriteBufferToKafkaProducer.cpp create mode 100644 src/Storages/MessageQueueSink.cpp create mode 100644 src/Storages/MessageQueueSink.h delete mode 100644 src/Storages/NATS/Buffer_fwd.h rename src/Storages/NATS/{ReadBufferFromNATSConsumer.cpp => NATSConsumer.cpp} (63%) rename src/Storages/NATS/{ReadBufferFromNATSConsumer.h => NATSConsumer.h} (84%) create mode 100644 src/Storages/NATS/NATSProducer.cpp create mode 100644 src/Storages/NATS/NATSProducer.h delete mode 100644 src/Storages/NATS/NATSSink.cpp delete mode 100644 src/Storages/NATS/NATSSink.h delete mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.cpp delete mode 100644 src/Storages/NATS/WriteBufferToNATSProducer.h delete mode 100644 src/Storages/RabbitMQ/Buffer_fwd.h rename src/Storages/RabbitMQ/{ReadBufferFromRabbitMQConsumer.cpp => RabbitMQConsumer.cpp} (78%) rename src/Storages/RabbitMQ/{ReadBufferFromRabbitMQConsumer.h => RabbitMQConsumer.h} (88%) rename src/Storages/RabbitMQ/{WriteBufferToRabbitMQProducer.cpp => RabbitMQProducer.cpp} (67%) rename src/Storages/RabbitMQ/{WriteBufferToRabbitMQProducer.h => RabbitMQProducer.h} (64%) delete mode 100644 src/Storages/RabbitMQ/RabbitMQSink.cpp delete mode 100644 src/Storages/RabbitMQ/RabbitMQSink.h diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 88a0d08ebbd..7de856716fb 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -34,7 +34,14 @@ SETTINGS [kafka_max_block_size = 0,] [kafka_skip_broken_messages = N,] [kafka_commit_every_batch = 0,] - [kafka_thread_per_consumer = 0] + [kafka_client_id = '',] + [kafka_poll_timeout_ms = 0,] + [kafka_poll_max_batch_size = 0,] + [kafka_flush_interval_ms = 0,] + [kafka_thread_per_consumer = 0,] + [kafka_handle_error_mode = 'default',] + [kafka_commit_on_select = false,] + [kafka_max_rows_per_message = 1]; ``` Required parameters: @@ -46,13 +53,20 @@ Required parameters: Optional parameters: -- `kafka_row_delimiter` — Delimiter character, which ends the message. +- `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. Default: `1`. 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. -- `kafka_max_block_size` — The maximum batch size (in messages) for poll (default: `max_block_size`). -- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). -- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`). -- `kafka_thread_per_consumer` — Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block). +- `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). +- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). Default: `0`. +- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block. Default: `0`. +- `kafka_client_id` — Client identifier. Empty by default. +- `kafka_poll_timeout_ms` — Timeout for single poll from Kafka. Default: [stream_poll_timeout_ms](../../../operations/settings/settings.md#stream_poll_timeout_ms). +- `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_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`. Examples: @@ -94,7 +108,7 @@ Do not use this method in new projects. If possible, switch old projects to the ``` sql Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format - [, kafka_row_delimiter, kafka_schema, kafka_num_consumers, kafka_skip_broken_messages]) + [, kafka_row_delimiter, kafka_schema, kafka_num_consumers, kafka_max_block_size, kafka_skip_broken_messages, kafka_commit_every_batch, kafka_client_id, kafka_poll_timeout_ms, kafka_poll_max_batch_size, kafka_flush_interval_ms, kafka_thread_per_consumer, kafka_handle_error_mode, kafka_commit_on_select, kafka_max_rows_per_message]); ``` @@ -193,6 +207,14 @@ Example: - `_headers.name` — Array of message's headers keys. - `_headers.value` — Array of message's headers values. +## Data formats support {#data-formats-support} + +Kafka engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse. +The number of rows in one Kafka message depends on whether the format is row-based or block-based: + +- For row-based formats the number of rows in one Kafka message can be controlled by setting `kafka_max_rows_per_message`. +- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size). + **See Also** - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 90b30dc8295..35ad9068be8 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -37,8 +37,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [nats_max_block_size = N,] [nats_flush_interval_ms = N,] [nats_username = 'user',] - [nats_password = 'password'] - [redis_password = 'clickhouse'] + [nats_password = 'password',] + [nats_token = 'clickhouse',] + [nats_startup_connect_tries = '5'] + [nats_max_rows_per_message = 1] ``` Required parameters: @@ -49,7 +51,7 @@ Required parameters: Optional parameters: -- `nats_row_delimiter` – Delimiter character, which ends the message. +- `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. @@ -57,11 +59,13 @@ Optional parameters: - `nats_reconnect_wait` – Amount of time in milliseconds to sleep between each reconnect attempt. Default: `5000`. - `nats_server_list` - Server list for connection. Can be specified to connect to NATS cluster. - `nats_skip_broken_messages` - NATS message parser tolerance to schema-incompatible messages per block. Default: `0`. If `nats_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). -- `nats_max_block_size` - Number of row collected by poll(s) for flushing data from NATS. -- `nats_flush_interval_ms` - Timeout for flushing data read from NATS. +- `nats_max_block_size` - Number of row collected by poll(s) for flushing data from NATS. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size). +- `nats_flush_interval_ms` - Timeout for flushing data read from NATS. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms). - `nats_username` - NATS username. - `nats_password` - NATS password. - `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`). SSL connection: @@ -159,6 +163,14 @@ If you want to change the target table by using `ALTER`, we recommend disabling ## Virtual Columns {#virtual-columns} -- `_subject` - NATS message subject. +- `_subject` - NATS message subject. + +## Data formats support {#data-formats-support} + +NATS engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse. +The number of rows in one NATS message depends on whether the format is row-based or block-based: + +- For row-based formats the number of rows in one NATS message can be controlled by setting `nats_max_rows_per_message`. +- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size). [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/nats/) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 9227e5cdbfd..2e5a45931f8 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -37,8 +37,16 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [rabbitmq_persistent = 0,] [rabbitmq_skip_broken_messages = N,] [rabbitmq_max_block_size = N,] - [rabbitmq_flush_interval_ms = N] - [rabbitmq_queue_settings_list = 'x-dead-letter-exchange=my-dlx,x-max-length=10,x-overflow=reject-publish'] + [rabbitmq_flush_interval_ms = N,] + [rabbitmq_queue_settings_list = 'x-dead-letter-exchange=my-dlx,x-max-length=10,x-overflow=reject-publish',] + [rabbitmq_queue_consume = false,] + [rabbitmq_address = '',] + [rabbitmq_vhost = '/',] + [rabbitmq_queue_consume = false,] + [rabbitmq_username = '',] + [rabbitmq_password = '',] + [rabbitmq_commit_on_select = false,] + [rabbitmq_max_rows_per_message = 1] ``` Required parameters: @@ -49,19 +57,27 @@ Required parameters: 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. -- `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. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. -- `rabbitmq_num_queues` – Total number of queues. Default: `1`. Increasing this number can significantly improve performance. -- `rabbitmq_queue_base` - Specify a hint for queue names. Use cases of this setting are described below. -- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. -- `rabbitmq_persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. -- `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. Default: `0`. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). -- `rabbitmq_max_block_size` -- `rabbitmq_flush_interval_ms` -- `rabbitmq_queue_settings_list` - allows to set RabbitMQ settings when creating a queue. Available settings: `x-max-length`, `x-max-length-bytes`, `x-message-ttl`, `x-expires`, `x-priority`, `x-max-priority`, `x-overflow`, `x-dead-letter-exchange`, `x-queue-type`. The `durable` setting is enabled automatically for the queue. +- `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`. +- `rabbitmq_queue_base` - Specify a hint for queue names. Use cases of this setting are described below. +- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. +- `rabbitmq_persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. +- `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). Default: `0`. +- `rabbitmq_max_block_size` - Number of row collected before flushing data from RabbitMQ. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size). +- `rabbitmq_flush_interval_ms` - Timeout for flushing data from RabbitMQ. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms). +- `rabbitmq_queue_settings_list` - allows to set RabbitMQ settings when creating a queue. Available settings: `x-max-length`, `x-max-length-bytes`, `x-message-ttl`, `x-expires`, `x-priority`, `x-max-priority`, `x-overflow`, `x-dead-letter-exchange`, `x-queue-type`. The `durable` setting is enabled automatically for the queue. +- `rabbitmq_address` - Address for connection. Use ether this setting or `rabbitmq_host_port`. +- `rabbitmq_vhost` - RabbitMQ vhost. Default: `'\'`. +- `rabbitmq_queue_consume` - Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings. Default: `false`. +- `rabbitmq_username` - RabbitMQ username. +- `rabbitmq_password` - RabbitMQ password. +- `rabbitmq_commit_on_select` - Commit messages when select query is made. Default: `false`. +- `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`. + SSL connection: @@ -166,11 +182,20 @@ Example: ## Virtual Columns {#virtual-columns} -- `_exchange_name` - RabbitMQ exchange name. -- `_channel_id` - ChannelID, on which consumer, who received the message, was declared. -- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel. -- `_redelivered` - `redelivered` flag of the message. -- `_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. +- `_exchange_name` - RabbitMQ exchange name. +- `_channel_id` - ChannelID, on which consumer, who received the message, was declared. +- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel. +- `_redelivered` - `redelivered` flag of the message. +- `_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. + +## Data formats support {#data-formats-support} + +RabbitMQ engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse. +The number of rows in one RabbitMQ message depends on whether the format is row-based or block-based: + +- For row-based formats the number of rows in one RabbitMQ message can be controlled by setting `rabbitmq_max_rows_per_message`. +- For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size). + [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/rabbitmq/) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dfcef4ae200..3a2a43d6821 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1011,6 +1011,12 @@ The default value is 7500. The smaller the value, the more often data is flushed into the table. Setting the value too low leads to poor performance. +## stream_poll_timeout_ms {#stream_poll_timeout_ms} + +Timeout for polling data from/to streaming storages. + +Default value: 500. + ## load_balancing {#settings-load_balancing} Specifies the algorithm of replicas selection that is used for distributed query processing. diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 87773041178..1560b11a0d3 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1393,7 +1393,7 @@ formatRow(format, x, y, ...) **Returned value** -- A formatted string. +- A formatted string. (for text formats it's usually terminated with the new line character). **Example** @@ -1417,6 +1417,74 @@ Result: └──────────────────────────────────┘ ``` +**Note**: If format contains suffix/prefix, it will be written in each row. + +**Example** + +Query: + +``` sql +SELECT formatRow('CustomSeparated', number, 'good') +FROM numbers(3) +SETTINGS format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='' +``` + +Result: + +``` text +┌─formatRow('CustomSeparated', number, 'good')─┐ +│ +0 good + │ +│ +1 good + │ +│ +2 good + │ +└──────────────────────────────────────────────┘ +``` + +Note: Only row-based formats are supported in this function. + +## formatRowNoNewline + +Converts arbitrary expressions into a string via given format. Differs from formatRow in that this function trims the last `\n` if any. + +**Syntax** + +``` sql +formatRowNoNewline(format, x, y, ...) +``` + +**Arguments** + +- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). +- `x`,`y`, ... — Expressions. + +**Returned value** + +- A formatted string. + +**Example** + +Query: + +``` sql +SELECT formatRowNoNewline('CSV', number, 'good') +FROM numbers(3); +``` + +Result: + +``` text +┌─formatRowNoNewline('CSV', number, 'good')─┐ +│ 0,"good" │ +│ 1,"good" │ +│ 2,"good" │ +└───────────────────────────────────────────┘ +``` + ## snowflakeToDateTime Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime](../data-types/datetime.md) format. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 36efd425077..c2beb55fee1 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1316,7 +1316,7 @@ formatRow(format, x, y, ...) **Возвращаемое значение** -- Отформатированная строка. +- Отформатированная строка. (в текстовых форматах обычно с завершающим переводом строки). **Пример** @@ -1340,6 +1340,74 @@ FROM numbers(3); └──────────────────────────────────┘ ``` +**Примечание**: если формат содержит префикс/суффикс, то он будет записан в каждой строке. + +**Пример** + +Запрос: + +``` sql +SELECT formatRow('CustomSeparated', number, 'good') +FROM numbers(3) +SETTINGS format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='' +``` + +Результат: + +``` text +┌─formatRow('CustomSeparated', number, 'good')─┐ +│ +0 good + │ +│ +1 good + │ +│ +2 good + │ +└──────────────────────────────────────────────┘ +``` + +**Примечание**: данная функция поддерживает только строковые форматы вывода. + +## formatRowNoNewline {#formatrownonewline} + +Преобразует произвольные выражения в строку заданного формата. Отличается от функции formatRow тем, что удаляет лишний перевод строки `\n` а конце, если он есть. + +**Синтаксис** + +``` sql +formatRowNoNewline(format, x, y, ...) +``` + +**Аргументы** + +- `format` — текстовый формат. Например, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). +- `x`,`y`, ... — выражения. + +**Возвращаемое значение** + +- Отформатированная строка (в текстовых форматах без завершающего перевода строки). + +**Пример** + +Запрос: + +``` sql +SELECT formatRowNoNewline('CSV', number, 'good') +FROM numbers(3); +``` + +Результат: + +``` text +┌─formatRowNoNewline('CSV', number, 'good')─┐ +│ 0,"good" │ +│ 1,"good" │ +│ 2,"good" │ +└───────────────────────────────────────────┘ +``` + ## snowflakeToDateTime {#snowflaketodatetime} Извлекает время из [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) в формате [DateTime](../data-types/datetime.md). diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index bfe651dd1af..9af96ae382e 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -318,7 +318,6 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( WriteBuffer & buf, const Block & sample, ContextPtr context, - WriteCallback callback, const std::optional & _format_settings) const { const auto & output_getter = getCreators(name).output_creator; @@ -332,9 +331,9 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting && !settings.output_format_json_array_of_rows) { - auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr + auto formatter_creator = [output_getter, sample, format_settings] (WriteBuffer & output) -> OutputFormatPtr { - return output_getter(output, sample, {callback}, format_settings); + return output_getter(output, sample, format_settings); }; ParallelFormattingOutputFormat::Params builder{buf, sample, formatter_creator, settings.max_threads}; @@ -347,7 +346,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( return format; } - return getOutputFormat(name, buf, sample, context, callback, _format_settings); + return getOutputFormat(name, buf, sample, context, _format_settings); } @@ -356,7 +355,6 @@ OutputFormatPtr FormatFactory::getOutputFormat( WriteBuffer & buf, const Block & sample, ContextPtr context, - WriteCallback callback, const std::optional & _format_settings) const { const auto & output_getter = getCreators(name).output_creator; @@ -366,15 +364,12 @@ OutputFormatPtr FormatFactory::getOutputFormat( if (context->hasQueryContext() && context->getSettingsRef().log_queries) context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); - RowOutputFormatParams params; - params.callback = std::move(callback); - auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - auto format = output_getter(buf, sample, params, format_settings); + auto format = output_getter(buf, sample, format_settings); /// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query. if (format_settings.enable_streaming) @@ -401,9 +396,8 @@ String FormatFactory::getContentType( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); Block empty_block; - RowOutputFormatParams empty_params; WriteBufferFromOwnString empty_buffer; - auto format = output_getter(empty_buffer, empty_block, empty_params, format_settings); + auto format = output_getter(empty_buffer, empty_block, format_settings); return format->getContentType(); } diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 6d76e2f913f..4d6d45b1eae 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -30,9 +30,9 @@ using ProcessorPtr = std::shared_ptr; class IInputFormat; class IOutputFormat; +class IRowOutputFormat; struct RowInputFormatParams; -struct RowOutputFormatParams; class ISchemaReader; class IExternalSchemaReader; @@ -41,6 +41,7 @@ using ExternalSchemaReaderPtr = std::shared_ptr; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; +using RowOutputFormatPtr = std::shared_ptr; template struct Memory; @@ -56,10 +57,6 @@ FormatSettings getFormatSettings(ContextPtr context, const T & settings); class FormatFactory final : private boost::noncopyable { public: - /// This callback allows to perform some additional actions after reading a single row. - /// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer. - using ReadCallback = std::function; - /** Fast reading data from buffer and save result to memory. * Reads at least `min_bytes` and some more until the end of the chunk, depends on the format. * If `max_rows` is non-zero the function also stops after reading the `max_rows` number of rows @@ -88,7 +85,6 @@ private: using OutputCreator = std::function; /// Some input formats can have non trivial readPrefix() and readSuffix(), @@ -152,7 +148,6 @@ public: WriteBuffer & buf, const Block & sample, ContextPtr context, - WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; OutputFormatPtr getOutputFormat( @@ -160,7 +155,6 @@ public: WriteBuffer & buf, const Block & sample, ContextPtr context, - WriteCallback callback = {}, const std::optional & _format_settings = std::nullopt) const; String getContentType( diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 0e05b59badf..c92b0b1cc71 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -37,7 +37,7 @@ namespace ProtobufReader::ProtobufReader(ReadBuffer & in_) - : in(in_) + : in(&in_) { } @@ -153,7 +153,7 @@ bool ProtobufReader::readFieldNumber(int & field_number_) { if (current_message_end == END_OF_FILE) { - if (unlikely(in.eof())) + if (unlikely(in->eof())) { current_message_end = cursor; return false; @@ -282,26 +282,26 @@ void ProtobufReader::readStringAndAppend(PaddedPODArray & str) void ProtobufReader::readBinary(void* data, size_t size) { - in.readStrict(reinterpret_cast(data), size); + in->readStrict(reinterpret_cast(data), size); cursor += size; } void ProtobufReader::ignore(UInt64 num_bytes) { - in.ignore(num_bytes); + in->ignore(num_bytes); cursor += num_bytes; } void ProtobufReader::ignoreAll() { - cursor += in.tryIgnore(std::numeric_limits::max()); + cursor += in->tryIgnore(std::numeric_limits::max()); } void ProtobufReader::moveCursorBackward(UInt64 num_bytes) { - if (in.offset() < num_bytes) + if (in->offset() < num_bytes) throwUnknownFormat(); - in.position() -= num_bytes; + in->position() -= num_bytes; cursor -= num_bytes; } @@ -313,7 +313,7 @@ UInt64 ProtobufReader::continueReadingVarint(UInt64 first_byte) # define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \ do \ { \ - in.readStrict(c); \ + in->readStrict(c); \ ++cursor; \ if constexpr ((byteNo) < 10) \ { \ @@ -352,7 +352,7 @@ void ProtobufReader::ignoreVarint() # define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \ do \ { \ - in.readStrict(c); \ + in->readStrict(c); \ ++cursor; \ if constexpr ((byteNo) < 10) \ { \ diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index a1a1ce7b2f1..9de45a4a37a 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -32,7 +32,9 @@ public: void readString(String & str); void readStringAndAppend(PaddedPODArray & str); - bool eof() const { return in.eof(); } + bool eof() const { return in->eof(); } + + void setReadBuffer(ReadBuffer & in_) { in = &in_; } private: void readBinary(void * data, size_t size); @@ -43,7 +45,7 @@ private: UInt64 ALWAYS_INLINE readVarint() { char c; - in.readStrict(c); + in->readStrict(c); UInt64 first_byte = static_cast(c); ++cursor; if (likely(!(c & 0x80))) @@ -56,7 +58,7 @@ private: void ignoreGroup(); [[noreturn]] void throwUnknownFormat() const; - ReadBuffer & in; + ReadBuffer * in; Int64 cursor = 0; bool root_message_has_length_delimiter = false; size_t current_message_level = 0; diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 567a2a9ee98..8a2643017c9 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -2508,6 +2508,11 @@ namespace writer->endMessage(/*with_length_delimiter = */ true); } + void reset() override + { + first_call_of_write_row = true; + } + void readRow(size_t row_num) override { if (first_call_of_read_row) diff --git a/src/Formats/ProtobufSerializer.h b/src/Formats/ProtobufSerializer.h index ebd136c1a82..d50f7e4956e 100644 --- a/src/Formats/ProtobufSerializer.h +++ b/src/Formats/ProtobufSerializer.h @@ -27,6 +27,7 @@ public: virtual void setColumns(const ColumnPtr * columns, size_t num_columns) = 0; virtual void writeRow(size_t row_num) = 0; virtual void finalizeWrite() {} + virtual void reset() {} virtual void setColumns(const MutableColumnPtr * columns, size_t num_columns) = 0; virtual void readRow(size_t row_num) = 0; diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 70655316c98..91f15e5c91d 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -27,23 +27,18 @@ namespace /** formatRow(, x, y, ...) is a function that allows you to use RowOutputFormat over * several columns to generate a string per row, such as CSV, TSV, JSONEachRow, etc. + * formatRowNoNewline(...) trims the newline character of each row. */ - +template class FunctionFormatRow : public IFunction { public: - static constexpr auto name = "formatRow"; + static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; FunctionFormatRow(const String & format_name_, ContextPtr context_) : format_name(format_name_), context(context_) { if (!FormatFactory::instance().getAllFormats().contains(format_name)) throw Exception("Unknown format " + format_name, ErrorCodes::UNKNOWN_FORMAT); - /// It's impossible to output separate rows in Avro format, because of specific - /// implementation (we cannot separate table schema and rows, rows are written - /// in our buffer in batches) - if (format_name == "Avro") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format Avro is not supported in function {}"); - } String getName() const override { return name; } @@ -65,24 +60,30 @@ public: arg_columns.insert(arguments[i]); materializeBlockInplace(arg_columns); auto format_settings = getFormatSettings(context); - /// For SQLInsert output format we should set max_batch_size settings to 1 so - /// each line will contain prefix INSERT INTO ... (otherwise only subset of columns - /// will contain it according to max_batch_size setting) - format_settings.sql_insert.max_batch_size = 1; - auto out = FormatFactory::instance().getOutputFormat(format_name, buffer, arg_columns, context, {}, format_settings); + auto out = FormatFactory::instance().getOutputFormat(format_name, buffer, arg_columns, context, format_settings); /// This function make sense only for row output formats. auto * row_output_format = dynamic_cast(out.get()); if (!row_output_format) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot turn rows into a {} format strings. {} function supports only row output formats", format_name, getName()); - auto & working_buf = row_output_format->getWriteBuffer(); auto columns = arg_columns.getColumns(); for (size_t i = 0; i != input_rows_count; ++i) { - row_output_format->write(columns, i); - writeChar('\0', working_buf); - offsets[i] = working_buf.count(); + row_output_format->writePrefixIfNot(); + row_output_format->writeRow(columns, i); + row_output_format->finalize(); + if constexpr (no_newline) + { + // replace '\n' with '\0' + if (buffer.position() != buffer.buffer().begin() && buffer.position()[-1] == '\n') + buffer.position()[-1] = '\0'; + } + else + writeChar('\0', buffer); + + offsets[i] = buffer.count(); + row_output_format->resetFormatter(); } return col_str; @@ -93,10 +94,11 @@ private: ContextPtr context; }; +template class FormatRowOverloadResolver : public IFunctionOverloadResolver { public: - static constexpr auto name = "formatRow"; + static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } explicit FormatRowOverloadResolver(ContextPtr context_) : context(context_) { } String getName() const override { return name; } @@ -114,7 +116,7 @@ public: if (const auto * name_col = checkAndGetColumnConst(arguments.at(0).column.get())) return std::make_unique( - std::make_shared(name_col->getValue(), context), + std::make_shared>(name_col->getValue(), context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); else @@ -131,7 +133,8 @@ private: REGISTER_FUNCTION(FormatRow) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 521acb6c8d6..525e11518bd 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -26,6 +26,7 @@ template class WriteBufferFromVector : public WriteBuffer { public: + using ValueType = typename VectorType::value_type; explicit WriteBufferFromVector(VectorType & vector_) : WriteBuffer(reinterpret_cast(vector_.data()), vector_.size()), vector(vector_) { @@ -50,9 +51,11 @@ public: bool isFinished() const { return finalized; } - void restart() + void restart(std::optional max_capacity = std::nullopt) { - if (vector.empty()) + if (max_capacity && vector.capacity() > max_capacity) + VectorType(initial_size, ValueType()).swap(vector); + else if (vector.empty()) vector.resize(initial_size); set(reinterpret_cast(vector.data()), vector.size()); finalized = false; @@ -68,8 +71,8 @@ private: { vector.resize( ((position() - reinterpret_cast(vector.data())) /// NOLINT - + sizeof(typename VectorType::value_type) - 1) /// Align up. - / sizeof(typename VectorType::value_type)); + + sizeof(ValueType) - 1) /// Align up. + / sizeof(ValueType)); /// Prevent further writes. set(nullptr, 0); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 86686b3eb13..833ccc6f7d1 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1199,7 +1199,6 @@ void executeQuery( compressed_buffer ? *compressed_buffer : *out_buf, materializeBlock(pipeline.getHeader()), context, - {}, output_format_settings); out->setAutoFlush(); diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index ce1a1b85678..939f9f35712 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -126,6 +126,7 @@ void IOutputFormat::finalize() writePrefixIfNot(); writeSuffixIfNot(); finalizeImpl(); + finalizeBuffers(); finalized = true; } diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index b53bd5fa1f8..6220ab253ee 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -76,17 +76,13 @@ public: void doNotWritePrefix() { need_write_prefix = false; } - virtual WriteBuffer & getWriteBuffer() const { return out; } - -protected: - friend class ParallelFormattingOutputFormat; - - virtual void consume(Chunk) = 0; - virtual void consumeTotals(Chunk) {} - virtual void consumeExtremes(Chunk) {} - virtual void finalizeImpl() {} - virtual void writePrefix() {} - virtual void writeSuffix() {} + void resetFormatter() + { + need_write_prefix = true; + need_write_suffix = true; + finalized = false; + resetFormatterImpl(); + } void writePrefixIfNot() { @@ -97,6 +93,9 @@ protected: } } +protected: + friend class ParallelFormattingOutputFormat; + void writeSuffixIfNot() { if (need_write_suffix) @@ -106,6 +105,15 @@ protected: } } + virtual void consume(Chunk) = 0; + virtual void consumeTotals(Chunk) {} + virtual void consumeExtremes(Chunk) {} + virtual void finalizeImpl() {} + virtual void finalizeBuffers() {} + virtual void writePrefix() {} + virtual void writeSuffix() {} + virtual void resetFormatterImpl() {} + /// Methods-helpers for parallel formatting. /// Set the number of rows that was already read in diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index c758697026a..ac44dbc0157 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -10,12 +10,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IRowOutputFormat::IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_) +IRowOutputFormat::IRowOutputFormat(const Block & header, WriteBuffer & out_) : IOutputFormat(header, out_) , num_columns(header.columns()) , types(header.getDataTypes()) , serializations(header.getSerializations()) - , params(params_) { } @@ -30,10 +29,6 @@ void IRowOutputFormat::consume(DB::Chunk chunk) writeRowBetweenDelimiter(); write(columns, row); - - if (params.callback) - params.callback(columns, row); - first_row = false; } } diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index d4769f294cc..3a648f00eba 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -9,14 +9,6 @@ namespace DB { -struct RowOutputFormatParams -{ - using WriteCallback = std::function; - - // Callback used to indicate that another row is written. - WriteCallback callback; -}; - class WriteBuffer; /** Output format that writes data row by row. @@ -24,16 +16,17 @@ class WriteBuffer; class IRowOutputFormat : public IOutputFormat { public: - using Params = RowOutputFormatParams; + /// Used to work with IRowOutputFormat explicitly. + void writeRow(const Columns & columns, size_t row_num) + { + first_row = false; + write(columns, row_num); + } - /** Write a row. - * Default implementation calls methods to write single values and delimiters - * (except delimiter between rows (writeRowBetweenDelimiter())). - */ - virtual void write(const Columns & columns, size_t row_num); + virtual void writeRowBetweenDelimiter() {} /// delimiter between rows protected: - IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_); + IRowOutputFormat(const Block & header, WriteBuffer & out_); void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override; void consumeExtremes(Chunk chunk) override; @@ -41,6 +34,11 @@ protected: virtual bool supportTotals() const { return false; } virtual bool supportExtremes() const { return false; } + /** Write a row. + * Default implementation calls methods to write single values and delimiters + * (except delimiter between rows (writeRowBetweenDelimiter())). + */ + virtual void write(const Columns & columns, size_t row_num); virtual void writeMinExtreme(const Columns & columns, size_t row_num); virtual void writeMaxExtreme(const Columns & columns, size_t row_num); virtual void writeTotals(const Columns & columns, size_t row_num); @@ -52,7 +50,6 @@ protected: virtual void writeFieldDelimiter() {} /// delimiter between values virtual void writeRowStartDelimiter() {} /// delimiter before each row virtual void writeRowEndDelimiter() {} /// delimiter after each row - virtual void writeRowBetweenDelimiter() {} /// delimiter between rows virtual void writePrefix() override {} /// delimiter before resultset virtual void writeSuffix() override {} /// delimiter after resultset virtual void writeBeforeTotals() {} @@ -66,7 +63,6 @@ protected: size_t num_columns; DataTypes types; Serializations serializations; - Params params; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 83eaefa8cf7..874709cc8d7 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -21,7 +21,6 @@ ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & : IOutputFormat(header_, out_) , stream{stream_} , format_settings{format_settings_} - , arrow_ostream{std::make_shared(out_)} { } @@ -65,8 +64,15 @@ void ArrowBlockOutputFormat::finalizeImpl() "Error while closing a table: {}", status.ToString()); } +void ArrowBlockOutputFormat::resetFormatterImpl() +{ + writer.reset(); + arrow_ostream.reset(); +} + void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr & schema) { + arrow_ostream = std::make_shared(out); arrow::Result> writer_status; // TODO: should we use arrow::ipc::IpcOptions::alignment? @@ -88,7 +94,6 @@ void registerOutputFormatArrow(FormatFactory & factory) "Arrow", [](WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, false, format_settings); @@ -99,7 +104,6 @@ void registerOutputFormatArrow(FormatFactory & factory) "ArrowStream", [](WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, true, format_settings); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index ce0bdab9bcb..3c977842625 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -27,6 +27,7 @@ public: private: void consume(Chunk) override; void finalizeImpl() override; + void resetFormatterImpl() override; void prepareWriter(const std::shared_ptr & schema); diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 0ec04c61321..3958489ecc1 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -902,11 +902,15 @@ AvroConfluentRowInputFormat::AvroConfluentRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(header_, in_, params_) , schema_registry(getConfluentSchemaRegistry(format_settings_)) - , input_stream(std::make_unique(*in)) - , decoder(avro::binaryDecoder()) , format_settings(format_settings_) { +} + +void AvroConfluentRowInputFormat::readPrefix() +{ + input_stream = std::make_unique(*in); + decoder = avro::binaryDecoder(); decoder->init(*input_stream); } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 3a029232420..4525d7d33b0 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -163,6 +163,7 @@ public: private: virtual bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readPrefix() override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index b63b1e7b9b1..0d56aea4c5c 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -436,8 +436,8 @@ static avro::Codec getCodec(const std::string & codec_name) } AvroRowOutputFormat::AvroRowOutputFormat( - WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_, params_) + WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) + : IRowOutputFormat(header_, out_) , settings(settings_) , serializer(header_.getColumnsWithTypeAndName(), std::make_unique(settings)) { @@ -471,67 +471,24 @@ void AvroRowOutputFormat::write(const Columns & columns, size_t row_num) file_writer_ptr->incr(); } -void AvroRowOutputFormat::writeSuffix() +void AvroRowOutputFormat::finalizeImpl() +{ + file_writer_ptr->close(); +} + +void AvroRowOutputFormat::resetFormatterImpl() { file_writer_ptr.reset(); } -void AvroRowOutputFormat::consume(DB::Chunk chunk) -{ - if (params.callback) - consumeImplWithCallback(std::move(chunk)); - else - consumeImpl(std::move(chunk)); -} - -void AvroRowOutputFormat::consumeImpl(DB::Chunk chunk) -{ - auto num_rows = chunk.getNumRows(); - const auto & columns = chunk.getColumns(); - - for (size_t row = 0; row < num_rows; ++row) - { - write(columns, row); - } - -} - -void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) -{ - auto num_rows = chunk.getNumRows(); - const auto & columns = chunk.getColumns(); - - for (size_t row = 0; row < num_rows;) - { - size_t current_row = row; - /// used by WriteBufferToKafkaProducer to obtain auxiliary data - /// from the starting row of a file - - writePrefixIfNot(); - for (size_t row_in_file = 0; - row_in_file < settings.avro.output_rows_in_file && row < num_rows; - ++row, ++row_in_file) - { - write(columns, row); - } - - file_writer_ptr->flush(); - writeSuffix(); - need_write_prefix = true; - - params.callback(columns, current_row); - } -} - void registerOutputFormatAvro(FormatFactory & factory) { factory.registerOutputFormat("Avro", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markFormatHasNoAppendSupport("Avro"); } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index 4834c8948b2..d7b15a95d26 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -46,27 +46,23 @@ private: class AvroRowOutputFormat final : public IRowOutputFormat { public: - AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_); + AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_); virtual ~AvroRowOutputFormat() override; - void consume(Chunk) override; String getName() const override { return "AvroRowOutputFormat"; } private: void write(const Columns & columns, size_t row_num) override; void writeField(const IColumn &, const ISerialization &, size_t) override {} virtual void writePrefix() override; - virtual void writeSuffix() override; + virtual void finalizeImpl() override; + virtual void resetFormatterImpl() override; void createFileWriter(); FormatSettings settings; AvroSerializer serializer; std::unique_ptr file_writer_ptr; - - void consumeImpl(Chunk); - void consumeImplWithCallback(Chunk); - }; } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index 60b722569a2..d862c3eb98d 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -10,8 +10,8 @@ namespace DB { -BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_) - : IRowOutputFormat(header, out_, params_), with_names(with_names_), with_types(with_types_) +BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_) + : IRowOutputFormat(header, out_), with_names(with_names_), with_types(with_types_) { } @@ -55,10 +55,9 @@ void registerOutputFormatRowBinary(FormatFactory & factory) factory.registerOutputFormat(format_name, [with_names, with_types]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, with_names, with_types, params); + return std::make_shared(buf, sample, with_names, with_types); }); factory.markOutputFormatSupportsParallelFormatting(format_name); }; diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index 40894608677..bdb25492aff 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class BinaryRowOutputFormat final: public IRowOutputFormat { public: - BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_); + BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_); String getName() const override { return "BinaryRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 700f6327672..f870ba34b1e 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -9,8 +9,8 @@ namespace DB { -CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -102,10 +102,9 @@ void registerOutputFormatCSV(FormatFactory & factory) factory.registerOutputFormat(format_name, [with_names, with_types]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, with_names, with_types, params, format_settings); + return std::make_shared(buf, sample, with_names, with_types, format_settings); }); factory.markOutputFormatSupportsParallelFormatting(format_name); }; diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 385c45bf0aa..d4ccaf3b3e2 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -20,7 +20,7 @@ public: /** with_names - output in the first line a header with column names * with_types - output in the next line header with the names of the types */ - CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types, const FormatSettings & format_settings_); String getName() const override { return "CSVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index fd33abfb587..3c21d9b0b4c 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -42,10 +42,9 @@ void CapnProtoOutputStream::write(const void * buffer, size_t size) CapnProtoRowOutputFormat::CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) + : IRowOutputFormat(header_, out_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) { schema = schema_parser.getMessageSchema(info); checkCapnProtoSchemaStructure(schema, getPort(PortKind::Main).getHeader(), format_settings.capn_proto.enum_comparing_mode); @@ -248,10 +247,9 @@ void registerOutputFormatCapnProto(FormatFactory & factory) factory.registerOutputFormat("CapnProto", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, FormatSchemaInfo(format_settings, "CapnProto", true), format_settings); + return std::make_shared(buf, sample, FormatSchemaInfo(format_settings, "CapnProto", true), format_settings); }); } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index d1f64838145..5cc7099d4c7 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -29,7 +29,6 @@ public: CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index 4c8cf19b923..7fa3d90ce81 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { CustomSeparatedRowOutputFormat::CustomSeparatedRowOutputFormat( - const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_) - : IRowOutputFormat(header_, out_, params_) + const Block & header_, WriteBuffer & out_, const FormatSettings & format_settings_, bool with_names_, bool with_types_) + : IRowOutputFormat(header_, out_) , with_names(with_names_) , with_types(with_types_) , format_settings(format_settings_) @@ -84,10 +84,9 @@ void registerOutputFormatCustomSeparated(FormatFactory & factory) factory.registerOutputFormat(format_name, [with_names, with_types]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(sample, buf, params, settings, with_names, with_types); + return std::make_shared(sample, buf, settings, with_names, with_types); }); factory.markOutputFormatSupportsParallelFormatting(format_name); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h index 0e04764b993..34fe1ce965c 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h @@ -11,7 +11,7 @@ class WriteBuffer; class CustomSeparatedRowOutputFormat final : public IRowOutputFormat { public: - CustomSeparatedRowOutputFormat(const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_); + CustomSeparatedRowOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & format_settings_, bool with_names_, bool with_types_); String getName() const override { return "CustomSeparatedRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index ff41352ff96..6f405dac1ff 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #if USE_HIVE @@ -31,12 +32,18 @@ HiveTextRowInputFormat::HiveTextRowInputFormat( HiveTextRowInputFormat::HiveTextRowInputFormat( const Block & header_, std::unique_ptr buf_, const Params & params_, const FormatSettings & format_settings_) : CSVRowInputFormat( - header_, *buf_, params_, true, false, format_settings_, std::make_unique(std::move(buf_), format_settings_)) + header_, *buf_, params_, true, false, format_settings_, std::make_unique(*buf_, format_settings_)), buf(std::move(buf_)) { } -HiveTextFormatReader::HiveTextFormatReader(std::unique_ptr buf_, const FormatSettings & format_settings_) - : CSVFormatReader(*buf_, format_settings_), buf(std::move(buf_)), input_field_names(format_settings_.hive_text.input_field_names) +void HiveTextRowInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + CSVRowInputFormat::setReadBuffer(*buf); +} + +HiveTextFormatReader::HiveTextFormatReader(PeekableReadBuffer & buf_, const FormatSettings & format_settings_) + : CSVFormatReader(buf_, format_settings_), buf(&buf_), input_field_names(format_settings_.hive_text.input_field_names) { } @@ -53,6 +60,12 @@ std::vector HiveTextFormatReader::readTypes() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "HiveTextRowInputFormat::readTypes is not implemented"); } +void HiveTextFormatReader::setReadBuffer(ReadBuffer & buf_) +{ + buf = assert_cast(&buf_); + CSVFormatReader::setReadBuffer(buf_); +} + void registerInputFormatHiveText(FormatFactory & factory) { factory.registerInputFormat( diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.h b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h index 61f5bf77b07..251486b247c 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.h +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h @@ -18,21 +18,27 @@ public: String getName() const override { return "HiveTextRowInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + private: HiveTextRowInputFormat( const Block & header_, std::unique_ptr buf_, const Params & params_, const FormatSettings & format_settings_); + + std::unique_ptr buf; }; class HiveTextFormatReader final : public CSVFormatReader { public: - HiveTextFormatReader(std::unique_ptr buf_, const FormatSettings & format_settings_); + HiveTextFormatReader(PeekableReadBuffer & buf_, const FormatSettings & format_settings_); std::vector readNames() override; std::vector readTypes() override; + void setReadBuffer(ReadBuffer & buf_) override; + private: - std::unique_ptr buf; + PeekableReadBuffer * buf; std::vector input_field_names; }; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormat.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormat.cpp index 9ced073aede..4a618d3a164 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormat.cpp @@ -34,7 +34,6 @@ void registerOutputFormatJSONColumns(FormatFactory & factory) factory.registerOutputFormat("JSONColumns", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings, format_settings.json.validate_utf8); diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormatBase.cpp index 66362d9eed4..87a87548a91 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockOutputFormatBase.cpp @@ -28,7 +28,6 @@ void JSONColumnsBlockOutputFormatBase::consume(Chunk chunk) void JSONColumnsBlockOutputFormatBase::writeSuffix() { - writeChunk(mono_chunk); mono_chunk.clear(); } diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp index f8b864ca65f..48499e5fec8 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp @@ -95,12 +95,18 @@ void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl() ostr->next(); } +void JSONColumnsWithMetadataBlockOutputFormat::resetFormatterImpl() +{ + JSONColumnsBlockOutputFormat::resetFormatterImpl(); + rows = 0; + statistics = Statistics(); +} + void registerOutputFormatJSONColumnsWithMetadata(FormatFactory & factory) { factory.registerOutputFormat("JSONColumnsWithMetadata", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h index f71bca50ddb..279995d6ebd 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h @@ -46,8 +46,6 @@ public: void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; } void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); } - WriteBuffer & getWriteBuffer() const override { return *ostr; } - protected: void consumeTotals(Chunk chunk) override; void consumeExtremes(Chunk chunk) override; @@ -55,6 +53,7 @@ protected: void writePrefix() override; void writeSuffix() override; void finalizeImpl() override; + void resetFormatterImpl() override; void writeChunkStart() override; void writeChunkEnd() override; diff --git a/src/Processors/Formats/Impl/JSONCompactColumnsBlockOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactColumnsBlockOutputFormat.cpp index 2ec9549fa96..bb746798f1e 100644 --- a/src/Processors/Formats/Impl/JSONCompactColumnsBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactColumnsBlockOutputFormat.cpp @@ -32,7 +32,6 @@ void registerOutputFormatJSONCompactColumns(FormatFactory & factory) factory.registerOutputFormat("JSONCompactColumns", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 6d74ca6d616..09332c309fa 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index 09444355b8f..a11f1e0da7b 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -11,12 +11,11 @@ namespace DB JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, bool with_types_, bool yield_strings_) - : RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_, params_) + : RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_) , settings(settings_) , with_names(with_names_) , with_types(with_types_) @@ -130,10 +129,9 @@ void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) factory.registerOutputFormat(format_name, [yield_strings, with_names, with_types]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, format_settings, with_names, with_types, yield_strings); + return std::make_shared(buf, sample, format_settings, with_names, with_types, yield_strings); }); factory.markOutputFormatSupportsParallelFormatting(format_name); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index 70242f5a089..bc5dc8de813 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -17,7 +17,6 @@ public: JSONCompactEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, bool with_types_, diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 47b79b71ae2..53ab69f797a 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -11,10 +11,9 @@ namespace DB JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_) - : JSONRowOutputFormat(out_, header, params_, settings_, yield_strings_) + : JSONRowOutputFormat(out_, header, settings_, yield_strings_) { } @@ -72,10 +71,9 @@ void registerOutputFormatJSONCompact(FormatFactory & factory) factory.registerOutputFormat("JSONCompact", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, format_settings, false); + return std::make_shared(buf, sample, format_settings, false); }); factory.markOutputFormatSupportsParallelFormatting("JSONCompact"); @@ -83,10 +81,9 @@ void registerOutputFormatJSONCompact(FormatFactory & factory) factory.registerOutputFormat("JSONCompactStrings", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, format_settings, true); + return std::make_shared(buf, sample, format_settings, true); }); factory.markOutputFormatSupportsParallelFormatting("JSONCompactStrings"); diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index 38123833f10..d10075f8834 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -19,7 +19,6 @@ public: JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index db5a027844b..cc55d7499d1 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -277,6 +277,7 @@ void JSONEachRowRowInputFormat::resetParser() read_columns.clear(); seen_columns.clear(); prev_positions.clear(); + allow_new_rows = true; } void JSONEachRowRowInputFormat::readPrefix() diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 849fd85c5f7..07913c2906c 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -12,9 +12,8 @@ namespace DB JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & settings_) - : RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_, params_), + : RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_), settings(settings_) { fields = JSONUtils::makeNamesValidJSONStrings(getPort(PortKind::Main).getHeader().getNames(), settings, settings.json.validate_utf8); @@ -81,13 +80,11 @@ void registerOutputFormatJSONEachRow(FormatFactory & factory) factory.registerOutputFormat(format, [serialize_as_strings]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & _format_settings) { FormatSettings settings = _format_settings; settings.json.serialize_as_strings = serialize_as_strings; - return std::make_shared(buf, sample, params, - settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting(format); }; diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 2a216275c24..62b8188cb4d 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -17,7 +17,6 @@ public: JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & settings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index fc643608bdb..988f99c5b7f 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -76,25 +76,21 @@ void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory) factory.registerOutputFormat("JSONEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & _format_settings) { FormatSettings settings = _format_settings; settings.json.serialize_as_strings = false; - return std::make_shared(buf, - sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.registerOutputFormat("JSONStringsEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & _format_settings) { FormatSettings settings = _format_settings; settings.json.serialize_as_strings = true; - return std::make_shared(buf, - sample, params, settings); + return std::make_shared(buf, sample, settings); }); } diff --git a/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.cpp index 6155efd4b63..a02199d6075 100644 --- a/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.cpp @@ -6,8 +6,8 @@ namespace DB { -JSONObjectEachRowRowOutputFormat::JSONObjectEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_) - : JSONEachRowRowOutputFormat(out_, header_, params_, settings_), field_index_for_object_name(getColumnIndexForJSONObjectEachRowObjectName(header_, settings_)) +JSONObjectEachRowRowOutputFormat::JSONObjectEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) + : JSONEachRowRowOutputFormat(out_, header_, settings_), field_index_for_object_name(getColumnIndexForJSONObjectEachRowObjectName(header_, settings_)) { } @@ -71,12 +71,11 @@ void registerOutputFormatJSONObjectEachRow(FormatFactory & factory) factory.registerOutputFormat("JSONObjectEachRow", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & _format_settings) { FormatSettings settings = _format_settings; settings.json.serialize_as_strings = false; - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("JSONObjectEachRow"); factory.markFormatHasNoAppendSupport("JSONObjectEachRow"); diff --git a/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.h index 19d9fe1aa53..1981931e91b 100644 --- a/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.h @@ -23,7 +23,6 @@ public: JSONObjectEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & settings_); String getName() const override { return "JSONObjectEachRowRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 98120abe8d8..be97af82400 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -11,10 +11,9 @@ namespace DB JSONRowOutputFormat::JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_) - : RowOutputFormatWithUTF8ValidationAdaptor(true, header, out_, params_), settings(settings_), yield_strings(yield_strings_) + : RowOutputFormatWithUTF8ValidationAdaptor(true, header, out_), settings(settings_), yield_strings(yield_strings_) { names = JSONUtils::makeNamesValidJSONStrings(header.getNames(), settings, true); } @@ -130,6 +129,13 @@ void JSONRowOutputFormat::finalizeImpl() ostr->next(); } +void JSONRowOutputFormat::resetFormatterImpl() +{ + RowOutputFormatWithUTF8ValidationAdaptor::resetFormatterImpl(); + row_count = 0; + statistics = Statistics(); +} + void JSONRowOutputFormat::onProgress(const Progress & value) { @@ -142,10 +148,9 @@ void registerOutputFormatJSON(FormatFactory & factory) factory.registerOutputFormat("JSON", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, format_settings, false); + return std::make_shared(buf, sample, format_settings, false); }); factory.markOutputFormatSupportsParallelFormatting("JSON"); @@ -154,10 +159,9 @@ void registerOutputFormatJSON(FormatFactory & factory) factory.registerOutputFormat("JSONStrings", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, params, format_settings, true); + return std::make_shared(buf, sample, format_settings, true); }); factory.markOutputFormatSupportsParallelFormatting("JSONStrings"); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 0975ff68b68..68c3f6bb62e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -19,7 +19,6 @@ public: JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_); @@ -35,8 +34,6 @@ public: statistics.rows_before_limit = rows_before_limit_; } - WriteBuffer & getWriteBuffer() const override { return *ostr; } - protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; @@ -59,6 +56,7 @@ protected: void writeAfterExtremes() override; void finalizeImpl() override; + void resetFormatterImpl() override; virtual void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index 282d5ade4f1..c5fa4ea890e 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -5,8 +5,8 @@ namespace DB { -MarkdownRowOutputFormat::MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) {} +MarkdownRowOutputFormat::MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), format_settings(format_settings_) {} void MarkdownRowOutputFormat::writePrefix() { @@ -71,10 +71,9 @@ void registerOutputFormatMarkdown(FormatFactory & factory) factory.registerOutputFormat("Markdown", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("Markdown"); diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h index 6b9598041f8..4edf2a69434 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -12,7 +12,7 @@ class ReadBuffer; class MarkdownRowOutputFormat final : public IRowOutputFormat { public: - MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "MarkdownRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index a470e193300..f81359d58a8 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -32,8 +32,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), packer(out_), format_settings(format_settings_) {} +MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), packer(out_), format_settings(format_settings_) {} void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num) { @@ -226,10 +226,9 @@ void registerOutputFormatMsgPack(FormatFactory & factory) factory.registerOutputFormat("MsgPack", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("MsgPack"); } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h index 81943b5f73c..61225cd2864 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h @@ -17,7 +17,7 @@ namespace DB class MsgPackRowOutputFormat final : public IRowOutputFormat { public: - MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "MsgPackRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 344c5c179db..fc94f7f0293 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -106,7 +106,6 @@ void registerOutputFormatMySQLWire(FormatFactory & factory) "MySQLWire", [](WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); } diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index a8e2ddf95e4..959b86ec051 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -115,7 +115,6 @@ void registerOutputFormatNative(FormatFactory & factory) factory.registerOutputFormat("Native", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings &) { return std::make_shared(buf, sample); diff --git a/src/Processors/Formats/Impl/NullFormat.cpp b/src/Processors/Formats/Impl/NullFormat.cpp index 6c457c71d14..59514be9abc 100644 --- a/src/Processors/Formats/Impl/NullFormat.cpp +++ b/src/Processors/Formats/Impl/NullFormat.cpp @@ -13,7 +13,6 @@ void registerOutputFormatNull(FormatFactory & factory) factory.registerOutputFormat("Null", []( WriteBuffer &, const Block & sample, - const RowOutputFormatParams &, const FormatSettings &) { return std::make_shared(sample); diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 7cf133e5739..43294355f2f 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -100,7 +100,7 @@ void ODBCDriver2BlockOutputFormat::writePrefix() void registerOutputFormatODBCDriver2(FormatFactory & factory) { factory.registerOutputFormat( - "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, const FormatSettings & format_settings) + "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); }); diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 4279a998a61..e1af4436789 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -515,6 +515,11 @@ void ORCBlockOutputFormat::finalizeImpl() writer->close(); } +void ORCBlockOutputFormat::resetFormatterImpl() +{ + writer.reset(); +} + void ORCBlockOutputFormat::prepareWriter() { const Block & header = getPort(PortKind::Main).getHeader(); @@ -531,7 +536,6 @@ void registerOutputFormatORC(FormatFactory & factory) factory.registerOutputFormat("ORC", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index 322778299ae..28837193d1a 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -44,6 +44,7 @@ public: private: void consume(Chunk chunk) override; void finalizeImpl() override; + void resetFormatterImpl() override; std::unique_ptr getORCType(const DataTypePtr & type); diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index fb58f5765c1..538b330fae5 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -20,6 +20,11 @@ namespace DB { +namespace ErrorCodes +{ + const extern int NOT_IMPLEMENTED; +} + /** * ORDER-PRESERVING parallel formatting of data formats. * The idea is similar to ParallelParsingInputFormat. @@ -167,6 +172,12 @@ private: void finalizeImpl() override; + void resetFormatterImpl() override + { + /// Resetting parallel formatting is not obvious and it's not used anywhere + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method resetFormatterImpl is not implemented for parallel formatting"); + } + InternalFormatterCreator internal_formatter_creator; /// Status to synchronize multiple threads. diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index c8e94311af5..e99b308b87b 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -74,13 +74,17 @@ void ParquetBlockOutputFormat::finalizeImpl() throw Exception{"Error while closing a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; } +void ParquetBlockOutputFormat::resetFormatterImpl() +{ + file_writer.reset(); +} + void registerOutputFormatParquet(FormatFactory & factory) { factory.registerOutputFormat( "Parquet", [](WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index c0421a4d99f..0518d9df77c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -36,6 +36,7 @@ public: private: void consume(Chunk) override; void finalizeImpl() override; + void resetFormatterImpl() override; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp index 0450051daf8..3fa126494d2 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp @@ -66,7 +66,6 @@ void registerOutputFormatPostgreSQLWire(FormatFactory & factory) "PostgreSQLWire", [](WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); } } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index c8ab22b123f..95c72d15fa9 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -54,6 +54,11 @@ protected: const IColumn & column, const ISerialization & serialization, size_t row_num, size_t value_width, size_t pad_to_width, bool align_right); + void resetFormatterImpl() override + { + total_rows = 0; + } + private: bool mono_block; /// For mono_block == true only @@ -68,7 +73,6 @@ void registerPrettyFormatWithNoEscapesAndMonoBlock(FormatFactory & factory, cons fact.registerOutputFormat(name, [no_escapes, mono_block]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & format_settings) { if (no_escapes) diff --git a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp index b7c5ef92328..d290280bdbf 100644 --- a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp @@ -82,9 +82,8 @@ static Float64 tryParseFloat(const String & s) PrometheusTextOutputFormat::PrometheusTextOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_) + : IRowOutputFormat(header_, out_) , string_serialization(DataTypeString().getDefaultSerialization()) , format_settings(format_settings_) { @@ -339,10 +338,9 @@ void registerOutputFormatPrometheus(FormatFactory & factory) factory.registerOutputFormat(FORMAT_NAME, []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); } diff --git a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.h b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.h index 69b8d10e56b..de87237c663 100644 --- a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.h +++ b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.h @@ -20,7 +20,6 @@ public: PrometheusTextOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "PrometheusTextOutputFormat"; } diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 73e8e7992d3..9777f2361a2 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -30,6 +30,12 @@ ProtobufListInputFormat::ProtobufListInputFormat( { } +void ProtobufListInputFormat::setReadBuffer(ReadBuffer & in_) +{ + reader->setReadBuffer(in_); + IRowInputFormat::setReadBuffer(in_); +} + bool ProtobufListInputFormat::readRow(MutableColumns & columns, RowReadExtension & row_read_extension) { if (reader->eof()) diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.h b/src/Processors/Formats/Impl/ProtobufListInputFormat.h index 7c8bfb9b443..ba2e8014878 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.h @@ -33,6 +33,8 @@ public: String getName() const override { return "ProtobufListInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + private: bool readRow(MutableColumns & columns, RowReadExtension & row_read_extension) override; diff --git a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp index e5320b37ae7..815b9ebb61d 100644 --- a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp @@ -13,10 +13,9 @@ namespace DB ProtobufListOutputFormat::ProtobufListOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & schema_info_, bool defaults_for_nullable_google_wrappers_) - : IRowOutputFormat(header_, out_, params_) + : IRowOutputFormat(header_, out_) , writer(std::make_unique(out)) , serializer(ProtobufSerializer::create( header_.getNames(), @@ -42,18 +41,21 @@ void ProtobufListOutputFormat::finalizeImpl() serializer->finalizeWrite(); } +void ProtobufListOutputFormat::resetFormatterImpl() +{ + serializer->reset(); +} + void registerOutputFormatProtobufList(FormatFactory & factory) { factory.registerOutputFormat( "ProtobufList", [](WriteBuffer & buf, const Block & header, - const RowOutputFormatParams & params, const FormatSettings & settings) { return std::make_shared( - buf, header, params, - FormatSchemaInfo(settings, "Protobuf", true), + buf, header, FormatSchemaInfo(settings, "Protobuf", true), settings.protobuf.output_nullables_with_google_wrappers); }); } diff --git a/src/Processors/Formats/Impl/ProtobufListOutputFormat.h b/src/Processors/Formats/Impl/ProtobufListOutputFormat.h index 7b3513bb7ed..d85018c0351 100644 --- a/src/Processors/Formats/Impl/ProtobufListOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListOutputFormat.h @@ -26,7 +26,6 @@ public: ProtobufListOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & schema_info_, bool defaults_for_nullable_google_wrappers_); @@ -39,6 +38,7 @@ private: void writeField(const IColumn &, const ISerialization &, size_t) override {} void finalizeImpl() override; + void resetFormatterImpl() override; std::unique_ptr writer; std::unique_ptr serializer; diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 3046b005fa8..40f6a2a54a7 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -44,6 +44,12 @@ bool ProtobufRowInputFormat::readRow(MutableColumns & columns, RowReadExtension return true; } +void ProtobufRowInputFormat::setReadBuffer(ReadBuffer & in_) +{ + reader->setReadBuffer(in_); + IRowInputFormat::setReadBuffer(in_); +} + bool ProtobufRowInputFormat::allowSyncAfterError() const { return true; diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 1747d090976..2e0ed49b768 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -38,6 +38,8 @@ public: String getName() const override { return "ProtobufRowInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + private: bool readRow(MutableColumns & columns, RowReadExtension & row_read_extension) override; bool allowSyncAfterError() const override; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 7d8a67bdfb6..3faeefbaabd 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -20,11 +20,10 @@ namespace ErrorCodes ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & schema_info_, const FormatSettings & settings_, bool with_length_delimiter_) - : IRowOutputFormat(header_, out_, params_) + : IRowOutputFormat(header_, out_) , writer(std::make_unique(out)) , serializer(ProtobufSerializer::create( header_.getNames(), @@ -59,14 +58,11 @@ void registerOutputFormatProtobuf(FormatFactory & factory) with_length_delimiter ? "Protobuf" : "ProtobufSingle", [with_length_delimiter](WriteBuffer & buf, const Block & header, - const RowOutputFormatParams & params, const FormatSettings & settings) { return std::make_shared( - buf, header, params, - FormatSchemaInfo(settings, "Protobuf", true), - settings, - with_length_delimiter); + buf, header, FormatSchemaInfo(settings, "Protobuf", true), + settings, with_length_delimiter); }); } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 01eaac288f5..f6ff5bae999 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -30,7 +30,6 @@ public: ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSchemaInfo & schema_info_, const FormatSettings & settings_, bool with_length_delimiter_); diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp index 1627bb0cad0..947bf2858a6 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -8,9 +8,8 @@ namespace DB RawBLOBRowOutputFormat::RawBLOBRowOutputFormat( WriteBuffer & out_, - const Block & header_, - const RowOutputFormatParams & params_) - : IRowOutputFormat(header_, out_, params_) + const Block & header_) + : IRowOutputFormat(header_, out_) { } @@ -30,10 +29,9 @@ void registerOutputFormatRawBLOB(FormatFactory & factory) factory.registerOutputFormat("RawBLOB", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, params); + return std::make_shared(buf, sample); }); } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h index f6c4f0a58ca..261103a0f6d 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -29,8 +29,7 @@ class RawBLOBRowOutputFormat final : public IRowOutputFormat public: RawBLOBRowOutputFormat( WriteBuffer & out_, - const Block & header_, - const RowOutputFormatParams & params_); + const Block & header_); String getName() const override { return "RawBLOBRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp index 46d81cb9af4..891bbb5ff12 100644 --- a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp @@ -5,8 +5,8 @@ namespace DB { -SQLInsertRowOutputFormat::SQLInsertRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), column_names(header_.getNames()), format_settings(format_settings_) +SQLInsertRowOutputFormat::SQLInsertRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), column_names(header_.getNames()), format_settings(format_settings_) { } @@ -88,16 +88,19 @@ void SQLInsertRowOutputFormat::writeSuffix() writeChar('\n', out); } +void SQLInsertRowOutputFormat::resetFormatterImpl() +{ + rows_in_line = 0; +} void registerOutputFormatSQLInsert(FormatFactory & factory) { factory.registerOutputFormat("SQLInsert", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); } diff --git a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.h b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.h index aaaf39a9e4d..8b87744ed7c 100644 --- a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.h +++ b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.h @@ -16,7 +16,6 @@ public: SQLInsertRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "SQLInsertRowOutputFormat"; } @@ -26,11 +25,12 @@ public: protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; - virtual void writeFieldDelimiter() override; - virtual void writeRowStartDelimiter() override; - virtual void writeRowEndDelimiter() override; - virtual void writeRowBetweenDelimiter() override; - virtual void writeSuffix() override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; + void writeSuffix() override; + void resetFormatterImpl() override; void printLineStart(); void printColumnNames(); diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index 9c26c0c9b4f..f47c03df7b9 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -7,8 +7,8 @@ namespace DB { -TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header, false, false, false, params_, format_settings_), fields(header.getNamesAndTypes()) +TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out_, header, false, false, false, format_settings_), fields(header.getNamesAndTypes()) { for (auto & field : fields) { @@ -45,10 +45,9 @@ void registerOutputFormatTSKV(FormatFactory & factory) factory.registerOutputFormat("TSKV", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("TSKV"); } diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 99144298f74..2d05c8c05d0 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -14,7 +14,7 @@ namespace DB class TSKVRowOutputFormat final : public TabSeparatedRowOutputFormat { public: - TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings); + TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings); String getName() const override { return "TSKVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index f95d875e3bd..1fd6f58ded6 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -12,9 +12,8 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( bool with_names_, bool with_types_, bool is_raw_, - const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), is_raw(is_raw_), format_settings(format_settings_) + : IRowOutputFormat(header_, out_), with_names(with_names_), with_types(with_types_), is_raw(is_raw_), format_settings(format_settings_) { } @@ -108,10 +107,9 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) factory.registerOutputFormat(format_name, [is_raw, with_names, with_types]( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, with_names, with_types, is_raw, params, settings); + return std::make_shared(buf, sample, with_names, with_types, is_raw, settings); }); factory.markOutputFormatSupportsParallelFormatting(format_name); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index bddb7590d8d..6d481bdeec4 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -25,7 +25,6 @@ public: bool with_names_, bool with_types_, bool is_raw_, - const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "TabSeparatedRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 0e7bdb259ac..11f53917b93 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -133,9 +133,6 @@ void TemplateBlockOutputFormat::writePrefix() void TemplateBlockOutputFormat::finalizeImpl() { - if (finalized) - return; - size_t parts = format.format_idx_to_column_idx.size(); auto outside_statistics = getOutsideStatistics(); if (outside_statistics) @@ -184,17 +181,19 @@ void TemplateBlockOutputFormat::finalizeImpl() } writeString(format.delimiters[i + 1], out); } - - finalized = true; } +void TemplateBlockOutputFormat::resetFormatterImpl() +{ + row_count = 0; + statistics = Statistics(); +} void registerOutputFormatTemplate(FormatFactory & factory) { factory.registerOutputFormat("Template", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams &, const FormatSettings & settings) { ParsedTemplateFormatString resultset_format; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 419fcac37c1..2424a237186 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -44,6 +44,7 @@ private: void consumeTotals(Chunk chunk) override { statistics.totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { statistics.extremes = std::move(chunk); } void finalizeImpl() override; + void resetFormatterImpl() override; void writeRow(const Chunk & chunk, size_t row_num); template void writeValue(U value, EscapingRule escaping_rule); diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index 785658c0fa2..09a5c69530d 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -267,6 +267,7 @@ void TemplateRowInputFormat::resetParser() void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_) { buf = std::make_unique(in_); + format_reader->setReadBuffer(*buf); IInputFormat::setReadBuffer(*buf); } diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index 275bbad9e66..0612bde0fa3 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp @@ -10,8 +10,8 @@ namespace DB { -ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) +ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), format_settings(format_settings_) { } @@ -46,10 +46,9 @@ void registerOutputFormatValues(FormatFactory & factory) factory.registerOutputFormat("Values", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("Values"); diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h index 76c0a1e7873..35840d7f0c9 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -15,7 +15,7 @@ class WriteBuffer; class ValuesRowOutputFormat final : public IRowOutputFormat { public: - ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "ValuesRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp index 19a88a97559..cfaefbe663b 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -11,8 +11,8 @@ namespace DB { VerticalRowOutputFormat::VerticalRowOutputFormat( - WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) + WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -160,10 +160,9 @@ void registerOutputFormatVertical(FormatFactory & factory) factory.registerOutputFormat("Vertical", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("Vertical"); diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index c3dbafd8b9b..5870c3503fc 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -18,7 +18,7 @@ class Context; class VerticalRowOutputFormat final : public IRowOutputFormat { public: - VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "VerticalRowOutputFormat"; } @@ -45,6 +45,11 @@ private: /// For totals and extremes. void writeSpecialRow(const Columns & columns, size_t row_num, const char * title); + void resetFormatterImpl() override + { + row_number = 0; + } + const FormatSettings format_settings; size_t field_number = 0; size_t row_number = 0; diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index f986f0492e0..0472c2c83f1 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -7,8 +7,8 @@ namespace DB { -XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : RowOutputFormatWithUTF8ValidationAdaptor(true, header_, out_, params_), fields(header_.getNamesAndTypes()), format_settings(format_settings_) +XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) + : RowOutputFormatWithUTF8ValidationAdaptor(true, header_, out_), fields(header_.getNamesAndTypes()), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); field_tag_names.resize(sample.columns()); @@ -207,6 +207,13 @@ void XMLRowOutputFormat::finalizeImpl() ostr->next(); } +void XMLRowOutputFormat::resetFormatterImpl() +{ + RowOutputFormatWithUTF8ValidationAdaptor::resetFormatterImpl(); + row_count = 0; + statistics = Statistics(); +} + void XMLRowOutputFormat::writeRowsBeforeLimitAtLeast() { if (statistics.applied_limit) @@ -238,10 +245,9 @@ void registerOutputFormatXML(FormatFactory & factory) factory.registerOutputFormat("XML", []( WriteBuffer & buf, const Block & sample, - const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, settings); + return std::make_shared(buf, sample, settings); }); factory.markOutputFormatSupportsParallelFormatting("XML"); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index b3bcb334083..a213f8f73c1 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -16,12 +16,10 @@ namespace DB class XMLRowOutputFormat final : public RowOutputFormatWithUTF8ValidationAdaptor { public: - XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "XMLRowOutputFormat"; } - WriteBuffer & getWriteBuffer() const override { return *ostr; } - private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowStartDelimiter() override; @@ -30,6 +28,7 @@ private: void writePrefix() override; void writeSuffix() override; void finalizeImpl() override; + void resetFormatterImpl() override; void writeMinExtreme(const Columns & columns, size_t row_num) override; void writeMaxExtreme(const Columns & columns, size_t row_num) override; diff --git a/src/Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h b/src/Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h index b3f1641bd00..8d8fb9ef0c6 100644 --- a/src/Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h +++ b/src/Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h @@ -40,6 +40,18 @@ public: this->out.next(); } + void finalizeBuffers() override + { + if (validating_ostr) + validating_ostr->finalize(); + } + + void resetFormatterImpl() override + { + validating_ostr = std::make_unique(this->out); + ostr = validating_ostr.get(); + } + protected: /// Point to validating_ostr or out from IOutputFormat, should be used in derived classes instead of out. WriteBuffer * ostr; @@ -50,7 +62,7 @@ private: }; using OutputFormatWithUTF8ValidationAdaptor = OutputFormatWithUTF8ValidationAdaptorBase; -using RowOutputFormatWithUTF8ValidationAdaptor = OutputFormatWithUTF8ValidationAdaptorBase; +using RowOutputFormatWithUTF8ValidationAdaptor = OutputFormatWithUTF8ValidationAdaptorBase; } diff --git a/src/Processors/examples/comma_separated_streams.cpp b/src/Processors/examples/comma_separated_streams.cpp index a8c925354d9..2ec5564f346 100644 --- a/src/Processors/examples/comma_separated_streams.cpp +++ b/src/Processors/examples/comma_separated_streams.cpp @@ -102,8 +102,7 @@ try auto pipeline = QueryPipeline(std::move(input_format)); auto reader = std::make_unique(pipeline); - RowOutputFormatParams out_params; - OutputFormatPtr output_format = std::make_shared(out_buf, sample, true, true, out_params, format_settings); + OutputFormatPtr output_format = std::make_shared(out_buf, sample, true, true, format_settings); Block res; while (reader->pull(res)) { diff --git a/src/Storages/IMessageProducer.cpp b/src/Storages/IMessageProducer.cpp new file mode 100644 index 00000000000..96c8f6fbd9b --- /dev/null +++ b/src/Storages/IMessageProducer.cpp @@ -0,0 +1,37 @@ +#include +#include +#include + +namespace DB +{ + +void ConcurrentMessageProducer::start(const ContextPtr & context) +{ + initialize(); + producing_task = context->getSchedulePool().createTask(getProducingTaskName(), [this] + { + producingTask(); + /// Notify that producing task is finished. + task_finished.store(true); + task_finished.notify_all(); + }); + producing_task->activateAndSchedule(); +} + +void ConcurrentMessageProducer::finish() +{ + LOG_DEBUG(&Poco::Logger::get("ConcurrentMessageProducer"), "finish"); + + /// We should execute finish logic only once. + if (finished.exchange(true)) + return; + + stopProducingTask(); + /// Wait until producing task is finished. + task_finished.wait(false); + producing_task->deactivate(); + finishImpl(); +} + + +} diff --git a/src/Storages/IMessageProducer.h b/src/Storages/IMessageProducer.h new file mode 100644 index 00000000000..53744c396de --- /dev/null +++ b/src/Storages/IMessageProducer.h @@ -0,0 +1,67 @@ +#pragma once + +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Interface for producing messages in streaming storages. +/// It's used in MessageQueueSink. +class IMessageProducer +{ +public: + /// Do some preparations. + virtual void start(const ContextPtr & context) = 0; + + /// Produce single message. + virtual void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) = 0; + + /// Finalize producer. + virtual void finish() = 0; + + virtual ~IMessageProducer() = default; +}; + +/// Implements interface for concurrent message producing. +class ConcurrentMessageProducer : public IMessageProducer +{ +public: + /// Create and schedule task in BackgroundSchedulePool that will produce messages. + void start(const ContextPtr & context) override; + + /// Stop producing task, wait for ot to finish and finalize. + void finish() override; + + /// In this method producer should not do any hard work and send message + /// to producing task, for example, by using ConcurrentBoundedQueue. + void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override = 0; + +protected: + /// Do some initialization before scheduling producing task. + virtual void initialize() {} + /// Tell producer to finish all work and stop producing task + virtual void stopProducingTask() = 0; + /// Do some finalization after producing task is stopped. + virtual void finishImpl() {} + + virtual String getProducingTaskName() const = 0; + /// Method that is called inside producing task, all producing wokr should be done here. + virtual void producingTask() = 0; + +private: + /// Flag, indicated that finish() method was called. + /// It's used to prevent doing finish logic more than once. + std::atomic finished = false; + /// Flag, indicated that producing task was finished. + /// It's used to wait until producing task is finished. + std::atomic task_finished = false; + + BackgroundSchedulePool::TaskHolder producing_task; +}; + + +} diff --git a/src/Storages/Kafka/Buffer_fwd.h b/src/Storages/Kafka/Buffer_fwd.h deleted file mode 100644 index 89e72454aa5..00000000000 --- a/src/Storages/Kafka/Buffer_fwd.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class ReadBufferFromKafkaConsumer; -class WriteBufferToKafkaProducer; - -using ConsumerBufferPtr = std::shared_ptr; -using ProducerBufferPtr = std::shared_ptr; - -} diff --git a/src/Storages/Kafka/KafkaBlockSink.cpp b/src/Storages/Kafka/KafkaBlockSink.cpp deleted file mode 100644 index 2c7cfffb9a1..00000000000 --- a/src/Storages/Kafka/KafkaBlockSink.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include - -#include -#include -#include - -namespace DB -{ - -KafkaSink::KafkaSink( - StorageKafka & storage_, - const StorageMetadataPtr & metadata_snapshot_, - const ContextPtr & context_) - : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) - , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , context(context_) -{ -} - -void KafkaSink::onStart() -{ - buffer = storage.createWriteBuffer(getHeader()); - - auto format_settings = getFormatSettings(context); - format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - - format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, - getHeader(), context, - [this](const Columns & columns, size_t row) - { - buffer->countRow(columns, row); - }, - format_settings); -} - -void KafkaSink::consume(Chunk chunk) -{ - format->write(getHeader().cloneWithColumns(chunk.detachColumns())); -} - -void KafkaSink::onFinish() -{ - if (format) - format->finalize(); - //flush(); - - if (buffer) - buffer->flush(); -} - -} diff --git a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp similarity index 89% rename from src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp rename to src/Storages/Kafka/KafkaConsumer.cpp index 8c45ce4be38..3c532a83c31 100644 --- a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -2,7 +2,8 @@ // should be defined before any instantiation #include -#include +#include +#include #include @@ -44,7 +45,7 @@ const std::size_t POLL_TIMEOUT_WO_ASSIGNMENT_MS = 50; const auto DRAIN_TIMEOUT_MS = 5000ms; -ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( +KafkaConsumer::KafkaConsumer( ConsumerPtr consumer_, Poco::Logger * log_, size_t max_batch_size, @@ -52,8 +53,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( bool intermediate_commit_, const std::atomic & stopped_, const Names & _topics) - : ReadBuffer(nullptr, 0) - , consumer(consumer_) + : consumer(consumer_) , log(log_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) @@ -127,7 +127,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( }); } -ReadBufferFromKafkaConsumer::~ReadBufferFromKafkaConsumer() +KafkaConsumer::~KafkaConsumer() { try { @@ -155,7 +155,7 @@ ReadBufferFromKafkaConsumer::~ReadBufferFromKafkaConsumer() // after unsubscribe, otherwise consumer will hang on destruction // see https://github.com/edenhill/librdkafka/issues/2077 // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. -void ReadBufferFromKafkaConsumer::drain() +void KafkaConsumer::drain() { auto start_time = std::chrono::steady_clock::now(); cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); @@ -194,7 +194,7 @@ void ReadBufferFromKafkaConsumer::drain() } -void ReadBufferFromKafkaConsumer::commit() +void KafkaConsumer::commit() { auto print_offsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) { @@ -279,7 +279,7 @@ void ReadBufferFromKafkaConsumer::commit() offsets_stored = 0; } -void ReadBufferFromKafkaConsumer::subscribe() +void KafkaConsumer::subscribe() { LOG_TRACE(log, "Already subscribed to topics: [{}]", boost::algorithm::join(consumer->get_subscription(), ", ")); @@ -313,22 +313,20 @@ void ReadBufferFromKafkaConsumer::subscribe() } cleanUnprocessed(); - allowed = false; // we can reset any flags (except of CONSUMER_STOPPED) before attempt of reading new block of data if (stalled_status != CONSUMER_STOPPED) stalled_status = NO_MESSAGES_RETURNED; } -void ReadBufferFromKafkaConsumer::cleanUnprocessed() +void KafkaConsumer::cleanUnprocessed() { messages.clear(); current = messages.begin(); - BufferBase::set(nullptr, 0, 0); offsets_stored = 0; } -void ReadBufferFromKafkaConsumer::unsubscribe() +void KafkaConsumer::unsubscribe() { LOG_TRACE(log, "Re-joining claimed consumer after failure"); cleanUnprocessed(); @@ -346,13 +344,13 @@ void ReadBufferFromKafkaConsumer::unsubscribe() } catch (const cppkafka::HandleException & e) { - LOG_ERROR(log, "Exception from ReadBufferFromKafkaConsumer::unsubscribe: {}", e.what()); + LOG_ERROR(log, "Exception from KafkaConsumer::unsubscribe: {}", e.what()); } } -void ReadBufferFromKafkaConsumer::resetToLastCommitted(const char * msg) +void KafkaConsumer::resetToLastCommitted(const char * msg) { if (!assignment.has_value() || assignment->empty()) { @@ -365,18 +363,15 @@ void ReadBufferFromKafkaConsumer::resetToLastCommitted(const char * msg) } // it do the poll when needed -bool ReadBufferFromKafkaConsumer::poll() +ReadBufferPtr KafkaConsumer::consume() { resetIfStopped(); if (polledDataUnusable()) - return false; + return nullptr; if (hasMorePolledMessages()) - { - allowed = true; - return true; - } + return getNextNonEmptyMessage(); if (intermediate_commit) commit(); @@ -401,7 +396,7 @@ bool ReadBufferFromKafkaConsumer::poll() resetIfStopped(); if (stalled_status == CONSUMER_STOPPED) { - return false; + return nullptr; } else if (stalled_status == REBALANCE_HAPPENED) { @@ -412,7 +407,7 @@ bool ReadBufferFromKafkaConsumer::poll() // otherwise we will continue polling from that position resetToLastCommitted("Rewind last poll after rebalance."); } - return false; + return nullptr; } if (new_messages.empty()) @@ -430,18 +425,18 @@ bool ReadBufferFromKafkaConsumer::poll() { LOG_WARNING(log, "Can't get assignment. Will keep trying."); stalled_status = NO_ASSIGNMENT; - return false; + return nullptr; } } else if (assignment->empty()) { LOG_TRACE(log, "Empty assignment."); - return false; + return nullptr; } else { LOG_TRACE(log, "Stalled"); - return false; + return nullptr; } } else @@ -459,17 +454,31 @@ bool ReadBufferFromKafkaConsumer::poll() { LOG_ERROR(log, "Only errors left"); stalled_status = ERRORS_RETURNED; - return false; + return nullptr; } ProfileEvents::increment(ProfileEvents::KafkaMessagesPolled, messages.size()); stalled_status = NOT_STALLED; - allowed = true; - return true; + return getNextNonEmptyMessage(); } -size_t ReadBufferFromKafkaConsumer::filterMessageErrors() +ReadBufferPtr KafkaConsumer::getNextNonEmptyMessage() +{ + if (current == messages.end()) + return nullptr; + + const auto * data = current->get_payload().get_data(); + size_t size = current->get_payload().get_size(); + ++current; + + if (data && size > 0) + return std::make_shared(data, size); + + return getNextNonEmptyMessage(); +} + +size_t KafkaConsumer::filterMessageErrors() { assert(current == messages.begin()); @@ -494,7 +503,7 @@ size_t ReadBufferFromKafkaConsumer::filterMessageErrors() return skipped; } -void ReadBufferFromKafkaConsumer::resetIfStopped() +void KafkaConsumer::resetIfStopped() { // we can react on stop only during fetching data // after block is formed (i.e. during copying data to MV / committing) we ignore stop attempts @@ -505,29 +514,8 @@ void ReadBufferFromKafkaConsumer::resetIfStopped() } } -/// Do commit messages implicitly after we processed the previous batch. -bool ReadBufferFromKafkaConsumer::nextImpl() -{ - if (!allowed || !hasMorePolledMessages()) - return false; - const auto * message_data = current->get_payload().get_data(); - size_t message_size = current->get_payload().get_size(); - - allowed = false; - ++current; - - /// If message is empty, return end of stream. - if (message_data == nullptr) - return false; - - /// const_cast is needed, because ReadBuffer works with non-const char *. - auto * new_position = reinterpret_cast(const_cast(message_data)); - BufferBase::set(new_position, message_size, 0); - return true; -} - -void ReadBufferFromKafkaConsumer::storeLastReadMessageOffset() +void KafkaConsumer::storeLastReadMessageOffset() { if (!isStalled()) { diff --git a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h similarity index 86% rename from src/Storages/Kafka/ReadBufferFromKafkaConsumer.h rename to src/Storages/Kafka/KafkaConsumer.h index f390d1c1330..3a00cfdf8c6 100644 --- a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -22,10 +22,10 @@ namespace DB using ConsumerPtr = std::shared_ptr; -class ReadBufferFromKafkaConsumer : public ReadBuffer +class KafkaConsumer { public: - ReadBufferFromKafkaConsumer( + KafkaConsumer( ConsumerPtr consumer_, Poco::Logger * log_, size_t max_batch_size, @@ -34,7 +34,8 @@ public: const std::atomic & stopped_, const Names & _topics ); - ~ReadBufferFromKafkaConsumer() override; + + ~KafkaConsumer(); void commit(); // Commit all processed messages. void subscribe(); // Subscribe internal consumer to topics. void unsubscribe(); // Unsubscribe internal consumer in case of failure. @@ -56,11 +57,9 @@ public: void storeLastReadMessageOffset(); void resetToLastCommitted(const char * msg); - // Polls batch of messages from Kafka or allows to read consecutive message by nextImpl - // returns true if there are some messages to process - // return false and sets stalled to false if there are no messages to process. - // additionally sets - bool poll(); + /// Polls batch of messages from Kafka and return read buffer containing the next message or + /// nullptr when there are no messages to process. + ReadBufferPtr consume(); // Return values for the message that's being read. String currentTopic() const { return current[-1].get_topic(); } @@ -94,7 +93,6 @@ private: StalledStatus stalled_status = NO_MESSAGES_RETURNED; bool intermediate_commit = true; - bool allowed = true; size_t waited_for_assignment = 0; const std::atomic & stopped; @@ -112,8 +110,7 @@ private: void resetIfStopped(); /// Return number of messages with an error. size_t filterMessageErrors(); - - bool nextImpl() override; + ReadBufferPtr getNextNonEmptyMessage(); }; } diff --git a/src/Storages/Kafka/KafkaProducer.cpp b/src/Storages/Kafka/KafkaProducer.cpp new file mode 100644 index 00000000000..84066228bef --- /dev/null +++ b/src/Storages/Kafka/KafkaProducer.cpp @@ -0,0 +1,139 @@ +#include "KafkaProducer.h" +#include "Core/Block.h" +#include "Columns/ColumnString.h" +#include "Columns/ColumnsNumber.h" + +#include + +namespace ProfileEvents +{ + extern const Event KafkaRowsWritten; + extern const Event KafkaProducerFlushes; + extern const Event KafkaMessagesProduced; + extern const Event KafkaProducerErrors; +} + +namespace DB +{ + +static const auto BATCH = 1000; + +namespace ErrorCodes +{ + const extern int LOGICAL_ERROR; +} + +KafkaProducer::KafkaProducer( + ProducerPtr producer_, const std::string & topic_, std::chrono::milliseconds poll_timeout, std::atomic & shutdown_called_, const Block & header) + : producer(producer_), topic(topic_), timeout(poll_timeout), shutdown_called(shutdown_called_), payloads(BATCH) +{ + if (header.has("_key")) + { + auto column_index = header.getPositionByName("_key"); + auto column_info = header.getByPosition(column_index); + if (isString(column_info.type)) + key_column_index = column_index; + // else ? (not sure it's a good place to report smth to user) + } + + if (header.has("_timestamp")) + { + auto column_index = header.getPositionByName("_timestamp"); + auto column_info = header.getByPosition(column_index); + if (isDateTime(column_info.type)) + timestamp_column_index = column_index; + } +} + +void KafkaProducer::produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) +{ + ProfileEvents::increment(ProfileEvents::KafkaRowsWritten, rows_in_message); + Payload payload; + payload.message = message; + + // Note: if it will be few rows per message - it will take the value from last row of block + if (key_column_index) + { + const auto & key_column = assert_cast(*columns[key_column_index.value()]); + payload.key = key_column.getDataAt(last_row).toString(); + } + + if (timestamp_column_index) + { + const auto & timestamp_column = assert_cast(*columns[timestamp_column_index.value()]); + payload.timestamp = std::chrono::seconds{timestamp_column.getElement(last_row)}; + } + + if (!payloads.push(std::move(payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); +} + + +void KafkaProducer::producingTask() +{ + cppkafka::MessageBuilder builder(topic); + while ((!payloads.isFinishedAndEmpty()) && !shutdown_called.load()) + { + Payload payload; + if (!payloads.pop(payload)) + break; + + builder.payload(payload.message); + + if (payload.key) + builder.key(cppkafka::Buffer(payload.key->data(), payload.key->size())); + + if (payload.timestamp) + builder.timestamp(*payload.timestamp); + + while (true) + { + try + { + producer->produce(builder); + } + catch (cppkafka::HandleException & e) + { + if (e.get_error() == RD_KAFKA_RESP_ERR__QUEUE_FULL) + { + producer->poll(timeout); + continue; + } + ProfileEvents::increment(ProfileEvents::KafkaProducerErrors); + throw; + } + ProfileEvents::increment(ProfileEvents::KafkaMessagesProduced); + break; + } + } +} + +void KafkaProducer::stopProducingTask() +{ + payloads.finish(); +} + +void KafkaProducer::finishImpl() +{ + // For unknown reason we may hit some internal timeout when inserting for the first time. + while (true) + { + try + { + producer->flush(timeout); + } + catch (cppkafka::HandleException & e) + { + if (e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) + continue; + + ProfileEvents::increment(ProfileEvents::KafkaProducerErrors); + throw; + } + + ProfileEvents::increment(ProfileEvents::KafkaProducerFlushes); + break; + } +} + +} diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/src/Storages/Kafka/KafkaProducer.h similarity index 51% rename from src/Storages/Kafka/WriteBufferToKafkaProducer.h rename to src/Storages/Kafka/KafkaProducer.h index 64b06571f0a..030724d0d66 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.h +++ b/src/Storages/Kafka/KafkaProducer.h @@ -1,8 +1,7 @@ #pragma once -#include #include - +#include #include #include @@ -14,45 +13,49 @@ namespace CurrentMetrics extern const Metric KafkaProducers; } - namespace DB { class Block; using ProducerPtr = std::shared_ptr; -class WriteBufferToKafkaProducer : public WriteBuffer +class KafkaProducer : public ConcurrentMessageProducer { public: - WriteBufferToKafkaProducer( + KafkaProducer( ProducerPtr producer_, const std::string & topic_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_, std::chrono::milliseconds poll_timeout, + std::atomic & shutdown_called_, const Block & header); - ~WriteBufferToKafkaProducer() override; - void countRow(const Columns & columns, size_t row); - void flush(); + void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override; private: - void nextImpl() override; - void addChunk(); - void reinitializeChunks(); + void stopProducingTask() override; + void finishImpl() override; + + String getProducingTaskName() const override { return "KafkaProducingTask"; } + + void producingTask() override; + + struct Payload + { + String message; + std::optional key; + std::optional timestamp; + }; + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaProducers}; ProducerPtr producer; const std::string topic; - const std::optional delim; - const size_t max_rows; - const size_t chunk_size; const std::chrono::milliseconds timeout; - size_t rows = 0; - std::list chunks; + std::atomic & shutdown_called; + std::optional key_column_index; std::optional timestamp_column_index; + ConcurrentBoundedQueue payloads; }; } diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 51e5ee47077..ae4b92a08e1 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -13,25 +13,25 @@ class ASTStorage; M(String, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ M(String, kafka_topic_list, "", "A list of Kafka topics.", 0) \ M(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ - M(String, kafka_client_id, "", "Client identifier.", 0) \ - M(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ - M(Bool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \ - /* default is stream_poll_timeout_ms */ \ - M(Milliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \ - /* default is min(max_block_size, kafka_max_block_size)*/ \ - M(UInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ - /* default is = max_insert_block_size / kafka_num_consumers */ \ - M(UInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ - /* default is stream_flush_interval_ms */ \ - M(Milliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 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 */ \ + M(UInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ + M(Bool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \ + M(String, kafka_client_id, "", "Client identifier.", 0) \ + /* default is stream_poll_timeout_ms */ \ + M(Milliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \ + M(UInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ + /* 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. Passible values: default, stream.", 0) \ + M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default, stream.", 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) \ /** TODO: */ /* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */ diff --git a/src/Storages/Kafka/KafkaSink.h b/src/Storages/Kafka/KafkaSink.h deleted file mode 100644 index 8e0801c7b5e..00000000000 --- a/src/Storages/Kafka/KafkaSink.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class IOutputFormat; -using IOutputFormatPtr = std::shared_ptr; - -class KafkaSink : public SinkToStorage -{ -public: - explicit KafkaSink( - StorageKafka & storage_, - const StorageMetadataPtr & metadata_snapshot_, - const ContextPtr & context_); - - void consume(Chunk chunk) override; - void onStart() override; - void onFinish() override; - String getName() const override { return "KafkaSink"; } - - ///void flush() override; - -private: - StorageKafka & storage; - StorageMetadataPtr metadata_snapshot; - const ContextPtr context; - ProducerBufferPtr buffer; - IOutputFormatPtr format; -}; - -} diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 38b404fe79e..db528adec79 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -1,7 +1,8 @@ #include #include -#include +#include +#include #include #include #include @@ -51,13 +52,13 @@ KafkaSource::KafkaSource( KafkaSource::~KafkaSource() { - if (!buffer) + if (!consumer) return; if (broken) - buffer->unsubscribe(); + consumer->unsubscribe(); - storage.pushReadBuffer(buffer); + storage.pushConsumer(consumer); } bool KafkaSource::checkTimeLimit() const @@ -75,15 +76,15 @@ bool KafkaSource::checkTimeLimit() const Chunk KafkaSource::generateImpl() { - if (!buffer) + if (!consumer) { auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); - buffer = storage.popReadBuffer(timeout); + consumer = storage.popConsumer(timeout); - if (!buffer) + if (!consumer) return {}; - buffer->subscribe(); + consumer->subscribe(); broken = true; } @@ -99,8 +100,9 @@ Chunk KafkaSource::generateImpl() auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM; + EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size); std::optional exception_message; size_t total_rows = 0; @@ -130,7 +132,7 @@ Chunk KafkaSource::generateImpl() else { e.addMessage("while parsing Kafka message (topic: {}, partition: {}, offset: {})'", - buffer->currentTopic(), buffer->currentPartition(), buffer->currentOffset()); + consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); throw; } }; @@ -141,32 +143,31 @@ Chunk KafkaSource::generateImpl() { size_t new_rows = 0; exception_message.reset(); - if (buffer->poll()) + if (auto buf = consumer->consume()) { - // poll provide one message at a time to the input_format ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); - new_rows = executor.execute(); + new_rows = executor.execute(*buf); } if (new_rows) { - // In read_kafka_message(), ReadBufferFromKafkaConsumer::nextImpl() + // In read_kafka_message(), KafkaConsumer::nextImpl() // will be called, that may make something unusable, i.e. clean - // ReadBufferFromKafkaConsumer::messages, which is accessed from - // ReadBufferFromKafkaConsumer::currentTopic() (and other helpers). - if (buffer->isStalled()) + // KafkaConsumer::messages, which is accessed from + // KafkaConsumer::currentTopic() (and other helpers). + if (consumer->isStalled()) throw Exception("Polled messages became unusable", ErrorCodes::LOGICAL_ERROR); ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); - buffer->storeLastReadMessageOffset(); + consumer->storeLastReadMessageOffset(); - auto topic = buffer->currentTopic(); - auto key = buffer->currentKey(); - auto offset = buffer->currentOffset(); - auto partition = buffer->currentPartition(); - auto timestamp_raw = buffer->currentTimestamp(); - auto header_list = buffer->currentHeaderList(); + auto topic = consumer->currentTopic(); + auto key = consumer->currentKey(); + auto offset = consumer->currentOffset(); + auto partition = consumer->currentPartition(); + auto timestamp_raw = consumer->currentTimestamp(); + auto header_list = consumer->currentHeaderList(); Array headers_names; Array headers_values; @@ -205,7 +206,7 @@ Chunk KafkaSource::generateImpl() { if (exception_message) { - auto payload = buffer->currentPayload(); + auto payload = consumer->currentPayload(); virtual_columns[8]->insert(payload); virtual_columns[9]->insert(*exception_message); } @@ -219,11 +220,11 @@ Chunk KafkaSource::generateImpl() total_rows = total_rows + new_rows; } - else if (buffer->polledDataUnusable()) + else if (consumer->polledDataUnusable()) { break; } - else if (buffer->isStalled()) + else if (consumer->isStalled()) { ++failed_poll_attempts; } @@ -232,11 +233,11 @@ Chunk KafkaSource::generateImpl() // We came here in case of tombstone (or sometimes zero-length) messages, and it is not something abnormal // TODO: it seems like in case of put_error_to_stream=true we may need to process those differently // currently we just skip them with note in logs. - buffer->storeLastReadMessageOffset(); - LOG_DEBUG(log, "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", buffer->currentTopic(), buffer->currentPartition(), buffer->currentOffset()); + consumer->storeLastReadMessageOffset(); + LOG_DEBUG(log, "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); } - if (!buffer->hasMorePolledMessages() + if (!consumer->hasMorePolledMessages() && (total_rows >= max_block_size || !checkTimeLimit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) { break; @@ -247,7 +248,7 @@ Chunk KafkaSource::generateImpl() { return {}; } - else if (buffer->polledDataUnusable()) + else if (consumer->polledDataUnusable()) { // the rows were counted already before by KafkaRowsRead, // so let's count the rows we ignore separately @@ -291,10 +292,10 @@ Chunk KafkaSource::generate() void KafkaSource::commit() { - if (!buffer) + if (!consumer) return; - buffer->commit(); + consumer->commit(); broken = false; } diff --git a/src/Storages/Kafka/KafkaSource.h b/src/Storages/Kafka/KafkaSource.h index 94e4224d8df..3d2edd4ebd1 100644 --- a/src/Storages/Kafka/KafkaSource.h +++ b/src/Storages/Kafka/KafkaSource.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include @@ -32,7 +32,7 @@ public: Chunk generate() override; void commit(); - bool isStalled() const { return !buffer || buffer->isStalled(); } + bool isStalled() const { return !consumer || consumer->isStalled(); } void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } @@ -44,7 +44,7 @@ private: Poco::Logger * log; UInt64 max_block_size; - ConsumerBufferPtr buffer; + KafkaConsumerPtr consumer; bool broken = true; bool is_finished = false; bool commit_in_suffix; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index fa52850fb39..2459cf53ddf 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -20,10 +20,10 @@ #include #include #include -#include +#include +#include #include #include -#include #include #include #include @@ -39,6 +39,7 @@ #include #include #include +#include #include "config_version.h" @@ -208,7 +209,7 @@ StorageKafka::StorageKafka( kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_) : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value)) , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) - , row_delimiter(kafka_settings->kafka_row_delimiter.value) + , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value)) , num_consumers(kafka_settings->kafka_num_consumers.value) , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) @@ -334,7 +335,28 @@ SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & if (topics.size() > 1) throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*this, metadata_snapshot, modified_context); + + cppkafka::Configuration conf; + conf.set("metadata.broker.list", brokers); + conf.set("client.id", client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + // TODO: fill required settings + updateConfiguration(conf); + + const Settings & settings = getContext()->getSettingsRef(); + size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); + const auto & header = metadata_snapshot->getSampleBlockNonMaterialized(); + + auto producer = std::make_unique( + std::make_shared(conf), topics[0], std::chrono::milliseconds(poll_timeout), shutdown_called, header); + + size_t max_rows = max_rows_per_message; + /// Need for backward compatibility. + if (format_name == "Avro" && local_context->getSettingsRef().output_format_avro_rows_in_file.changed) + max_rows = local_context->getSettingsRef().output_format_avro_rows_in_file.value; + return std::make_shared( + header, getFormatName(), max_rows, std::move(producer), getName(), modified_context); } @@ -344,7 +366,7 @@ void StorageKafka::startup() { try { - pushReadBuffer(createReadBuffer(i)); + pushConsumer(createConsumer(i)); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -374,29 +396,29 @@ void StorageKafka::shutdown() LOG_TRACE(log, "Closing consumers"); for (size_t i = 0; i < num_created_consumers; ++i) - auto buffer = popReadBuffer(); + auto consumer = popConsumer(); LOG_TRACE(log, "Consumers closed"); rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS); } -void StorageKafka::pushReadBuffer(ConsumerBufferPtr buffer) +void StorageKafka::pushConsumer(KafkaConsumerPtr consumer) { std::lock_guard lock(mutex); - buffers.push_back(buffer); + consumers.push_back(consumer); semaphore.set(); CurrentMetrics::sub(CurrentMetrics::KafkaConsumersInUse, 1); } -ConsumerBufferPtr StorageKafka::popReadBuffer() +KafkaConsumerPtr StorageKafka::popConsumer() { - return popReadBuffer(std::chrono::milliseconds::zero()); + return popConsumer(std::chrono::milliseconds::zero()); } -ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) +KafkaConsumerPtr StorageKafka::popConsumer(std::chrono::milliseconds timeout) { // Wait for the first free buffer if (timeout == std::chrono::milliseconds::zero()) @@ -409,32 +431,14 @@ ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) // Take the first available buffer from the list std::lock_guard lock(mutex); - auto buffer = buffers.back(); - buffers.pop_back(); + auto consumer = consumers.back(); + consumers.pop_back(); CurrentMetrics::add(CurrentMetrics::KafkaConsumersInUse, 1); - return buffer; -} - -ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) -{ - cppkafka::Configuration conf; - conf.set("metadata.broker.list", brokers); - conf.set("client.id", client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - // TODO: fill required settings - updateConfiguration(conf); - - auto producer = std::make_shared(conf); - const Settings & settings = getContext()->getSettingsRef(); - size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); - - return std::make_shared( - producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024, std::chrono::milliseconds(poll_timeout), header); + return consumer; } -ConsumerBufferPtr StorageKafka::createReadBuffer(size_t consumer_number) +KafkaConsumerPtr StorageKafka::createConsumer(size_t consumer_number) { cppkafka::Configuration conf; @@ -466,16 +470,16 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(size_t consumer_number) conf.set("enable.partition.eof", "false"); // Ignore EOF messages // Create a consumer and subscribe to topics - auto consumer = std::make_shared(conf); - consumer->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); + auto consumer_impl = std::make_shared(conf); + consumer_impl->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. if (thread_per_consumer) { auto& stream_cancelled = tasks[consumer_number]->stream_cancelled; - return std::make_shared(consumer, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); + return std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); } - return std::make_shared(consumer, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, tasks.back()->stream_cancelled, topics); + return std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, tasks.back()->stream_cancelled, topics); } size_t StorageKafka::getMaxBlockSize() const @@ -837,6 +841,13 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) } #undef CHECK_KAFKA_STORAGE_ARGUMENT diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 77bad6e17a9..fa99c5258b9 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -24,6 +24,8 @@ namespace DB struct StorageKafkaInterceptors; +using KafkaConsumerPtr = std::shared_ptr; + /** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, * or as a basic building block for creating pipelines with a continuous insertion / ETL. */ @@ -60,11 +62,12 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; - void pushReadBuffer(ConsumerBufferPtr buf); - ConsumerBufferPtr popReadBuffer(); - ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + /// We want to control the number of rows in a chunk inserted into Kafka + bool prefersLargeBlocks() const override { return false; } - ProducerBufferPtr createWriteBuffer(const Block & header); + void pushConsumer(KafkaConsumerPtr consumer); + KafkaConsumerPtr popConsumer(); + KafkaConsumerPtr popConsumer(std::chrono::milliseconds timeout); const auto & getFormatName() const { return format_name; } @@ -80,7 +83,7 @@ private: const String group; const String client_id; const String format_name; - const char row_delimiter; /// optional row delimiter for generating char delimited stream in order to make various input stream parsers happy. + const size_t max_rows_per_message; const String schema_name; const size_t num_consumers; /// total number of consumers Poco::Logger * log; @@ -94,7 +97,7 @@ private: /// In this case we still need to be able to shutdown() properly. size_t num_created_consumers = 0; /// number of actually created consumers. - std::vector buffers; /// available buffers for Kafka consumers + std::vector consumers; /// available consumers std::mutex mutex; @@ -115,11 +118,13 @@ private: std::list> thread_statuses; SettingsChanges createSettingsAdjustments(); - ConsumerBufferPtr createReadBuffer(size_t consumer_number); + KafkaConsumerPtr createConsumer(size_t consumer_number); /// If named_collection is specified. String collection_name; + std::atomic shutdown_called = false; + // Update Kafka configuration with values from CH user configuration. void updateConfiguration(cppkafka::Configuration & conf); String getConfigPrefix() const; diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp deleted file mode 100644 index 28877864e16..00000000000 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ /dev/null @@ -1,175 +0,0 @@ -#include "WriteBufferToKafkaProducer.h" -#include "Core/Block.h" -#include "Columns/ColumnString.h" -#include "Columns/ColumnsNumber.h" - -#include - -namespace ProfileEvents -{ - extern const Event KafkaRowsWritten; - extern const Event KafkaProducerFlushes; - extern const Event KafkaMessagesProduced; - extern const Event KafkaProducerErrors; -} - -namespace DB -{ -WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( - ProducerPtr producer_, - const std::string & topic_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_, - std::chrono::milliseconds poll_timeout, - const Block & header - ) - : WriteBuffer(nullptr, 0) - , producer(producer_) - , topic(topic_) - , delim(delimiter) - , max_rows(rows_per_message) - , chunk_size(chunk_size_) - , timeout(poll_timeout) -{ - if (header.has("_key")) - { - auto column_index = header.getPositionByName("_key"); - auto column_info = header.getByPosition(column_index); - if (isString(column_info.type)) - { - key_column_index = column_index; - } - // else ? (not sure it's a good place to report smth to user) - } - - if (header.has("_timestamp")) - { - auto column_index = header.getPositionByName("_timestamp"); - auto column_info = header.getByPosition(column_index); - if (isDateTime(column_info.type)) - { - timestamp_column_index = column_index; - } - } - - reinitializeChunks(); -} - -WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() -{ - assert(rows == 0); -} - -void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t current_row) -{ - ProfileEvents::increment(ProfileEvents::KafkaRowsWritten); - - if (++rows % max_rows == 0) - { - const std::string & last_chunk = chunks.back(); - size_t last_chunk_size = offset(); - - // if last character of last chunk is delimiter - we don't need it - if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) - --last_chunk_size; - - std::string payload; - payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); - - // concat all chunks except the last one - for (auto i = chunks.begin(), e = --chunks.end(); i != e; ++i) - payload.append(*i); - - // add last one - payload.append(last_chunk, 0, last_chunk_size); - - cppkafka::MessageBuilder builder(topic); - builder.payload(payload); - - // Note: if it will be few rows per message - it will take the value from last row of block - if (key_column_index) - { - const auto & key_column = assert_cast(*columns[key_column_index.value()]); - const auto key_data = key_column.getDataAt(current_row); - builder.key(cppkafka::Buffer(key_data.data, key_data.size)); - } - - if (timestamp_column_index) - { - const auto & timestamp_column = assert_cast(*columns[timestamp_column_index.value()]); - const auto timestamp = std::chrono::seconds{timestamp_column.getElement(current_row)}; - builder.timestamp(timestamp); - } - - while (true) - { - try - { - producer->produce(builder); - } - catch (cppkafka::HandleException & e) - { - if (e.get_error() == RD_KAFKA_RESP_ERR__QUEUE_FULL) - { - producer->poll(timeout); - continue; - } - ProfileEvents::increment(ProfileEvents::KafkaProducerErrors); - throw; - } - ProfileEvents::increment(ProfileEvents::KafkaMessagesProduced); - - break; - } - - reinitializeChunks(); - } -} - -void WriteBufferToKafkaProducer::flush() -{ - // For unknown reason we may hit some internal timeout when inserting for the first time. - while (true) - { - try - { - producer->flush(timeout); - } - catch (cppkafka::HandleException & e) - { - if (e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) - continue; - - ProfileEvents::increment(ProfileEvents::KafkaProducerErrors); - throw; - } - - ProfileEvents::increment(ProfileEvents::KafkaProducerFlushes); - break; - } -} - -void WriteBufferToKafkaProducer::nextImpl() -{ - addChunk(); -} - -void WriteBufferToKafkaProducer::addChunk() -{ - chunks.push_back(std::string()); - chunks.back().resize(chunk_size); - set(chunks.back().data(), chunk_size); -} - -void WriteBufferToKafkaProducer::reinitializeChunks() -{ - rows = 0; - chunks.clear(); - /// We cannot leave the buffer in the undefined state (i.e. without any - /// underlying buffer), since in this case the WriteBuffeR::next() will - /// not call our nextImpl() (due to available() == 0) - addChunk(); -} - -} diff --git a/src/Storages/MeiliSearch/SinkMeiliSearch.cpp b/src/Storages/MeiliSearch/SinkMeiliSearch.cpp index 2c843639669..e04275dfe50 100644 --- a/src/Storages/MeiliSearch/SinkMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/SinkMeiliSearch.cpp @@ -42,7 +42,7 @@ void SinkMeiliSearch::writeBlockData(const Block & block) const FormatSettings settings = getFormatSettings(local_context); settings.json.quote_64bit_integers = false; WriteBufferFromOwnString buf; - auto writer = FormatFactory::instance().getOutputFormat("JSON", buf, sample_block, local_context, {}, settings); + auto writer = FormatFactory::instance().getOutputFormat("JSON", buf, sample_block, local_context, settings); writer->write(block); writer->flush(); writer->finalize(); diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp new file mode 100644 index 00000000000..76dafb05e4d --- /dev/null +++ b/src/Storages/MessageQueueSink.cpp @@ -0,0 +1,78 @@ +#include +#include +#include +#include + +namespace DB +{ + +MessageQueueSink::MessageQueueSink( + const Block & header, + const String & format_name_, + size_t max_rows_per_message_, + std::unique_ptr producer_, + const String & storage_name_, + const ContextPtr & context_) + : SinkToStorage(header), format_name(format_name_), max_rows_per_message(max_rows_per_message_), producer(std::move(producer_)), storage_name(storage_name_), context(context_) +{ +} + +void MessageQueueSink::onStart() +{ + initialize(); + producer->start(context); + + buffer = std::make_unique(); + + auto format_settings = getFormatSettings(context); + format_settings.protobuf.allow_multiple_rows_without_delimiter = true; + + format = FormatFactory::instance().getOutputFormat(format_name, *buffer, getHeader(), context, format_settings); + row_format = dynamic_cast(format.get()); +} + +void MessageQueueSink::onFinish() +{ + producer->finish(); +} + +void MessageQueueSink::consume(Chunk chunk) +{ + const auto & columns = chunk.getColumns(); + if (columns.empty()) + return; + + if (row_format) + { + size_t row = 0; + while (row < chunk.getNumRows()) + { + row_format->writePrefixIfNot(); + size_t i = 0; + for (; i < max_rows_per_message && row < chunk.getNumRows(); ++i, ++row) + { + if (i != 0) + row_format->writeRowBetweenDelimiter(); + row_format->writeRow(columns, row); + } + row_format->finalize(); + row_format->resetFormatter(); + producer->produce(buffer->str(), i, columns, row - 1); + /// Reallocate buffer if it's capacity is large then DBMS_DEFAULT_BUFFER_SIZE, + /// because most likely in this case we serialized abnormally large row + /// and won't need this large allocated buffer anymore. + buffer->restart(DBMS_DEFAULT_BUFFER_SIZE); + } + } + else + { + format->write(getHeader().cloneWithColumns(chunk.detachColumns())); + format->finalize(); + producer->produce(buffer->str(), chunk.getNumRows(), columns, chunk.getNumRows() - 1); + format->resetFormatter(); + buffer->restart(); + } +} + + +} diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h new file mode 100644 index 00000000000..3c5dc988fde --- /dev/null +++ b/src/Storages/MessageQueueSink.h @@ -0,0 +1,66 @@ +#pragma once + +#pragma once + +#include +#include +#include + +namespace DB +{ + +class IOutputFormat; +class IRowOutputFormat; +using IOutputFormatPtr = std::shared_ptr; + + +/// Storage sink for streaming engines like Kafka/RabbitMQ/NATS. +/// It implements formatting input data into messages. +/// For row-based formats like TSV, CSV, JSONEachRow, etc, each message +/// contains from 1 to max_rows_per_message rows. +/// For block-based formats like Native, Arrow, Parquet, the whole block is formatted into one message. +/// Each message is created independently, so it contains all format +/// prefixes/suffixes and can fully parsed back by corresponding input format. +/// After formatting, created message is propagated to IMessageProducer::produce() method. +/// To use MessageQueueSink for specific streaming engine, you should implement +/// IMessageProducer for it. +class MessageQueueSink : public SinkToStorage +{ +public: + MessageQueueSink( + const Block & header, + const String & format_name_, + size_t max_rows_per_message_, + std::unique_ptr producer_, + const String & storage_name_, + const ContextPtr & context_); + + String getName() const override { return storage_name + "Sink"; } + + void consume(Chunk chunk) override; + + void onStart() override; + void onFinish() override; + void onCancel() override { onFinish(); } + void onException() override { onFinish(); } + +protected: + /// Do some specific initialization before consuming data. + virtual void initialize() {} + +private: + const String format_name; + size_t max_rows_per_message; + + std::unique_ptr buffer; + IOutputFormatPtr format; + IRowOutputFormat * row_format; + std::unique_ptr producer; + + const String storage_name; + +protected: + const ContextPtr context; +}; + +} diff --git a/src/Storages/NATS/Buffer_fwd.h b/src/Storages/NATS/Buffer_fwd.h deleted file mode 100644 index 3eb52314a79..00000000000 --- a/src/Storages/NATS/Buffer_fwd.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class ReadBufferFromNATSConsumer; -using ConsumerBufferPtr = std::shared_ptr; - -class WriteBufferToNATSProducer; -using ProducerBufferPtr = std::shared_ptr; - -} diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index c699f859446..b49070473b2 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp b/src/Storages/NATS/NATSConsumer.cpp similarity index 63% rename from src/Storages/NATS/ReadBufferFromNATSConsumer.cpp rename to src/Storages/NATS/NATSConsumer.cpp index fa6e60ac213..c7b40973b72 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.cpp +++ b/src/Storages/NATS/NATSConsumer.cpp @@ -2,9 +2,8 @@ #include #include #include -#include -#include -#include +#include +#include #include "Poco/Timer.h" #include @@ -17,28 +16,25 @@ namespace ErrorCodes extern const int CANNOT_CONNECT_NATS; } -ReadBufferFromNATSConsumer::ReadBufferFromNATSConsumer( +NATSConsumer::NATSConsumer( std::shared_ptr connection_, StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, Poco::Logger * log_, - char row_delimiter_, uint32_t queue_size_, const std::atomic & stopped_) - : ReadBuffer(nullptr, 0) - , connection(connection_) + : connection(connection_) , storage(storage_) , subjects(subjects_) , log(log_) - , row_delimiter(row_delimiter_) , stopped(stopped_) , queue_name(subscribe_queue_name) , received(queue_size_) { } -void ReadBufferFromNATSConsumer::subscribe() +void NATSConsumer::subscribe() { if (subscribed) return; @@ -62,49 +58,38 @@ void ReadBufferFromNATSConsumer::subscribe() subscribed = true; } -void ReadBufferFromNATSConsumer::unsubscribe() +void NATSConsumer::unsubscribe() { for (const auto & subscription : subscriptions) natsSubscription_Unsubscribe(subscription.get()); } -bool ReadBufferFromNATSConsumer::nextImpl() +ReadBufferPtr NATSConsumer::consume() { - if (stopped || !allowed) - return false; + if (stopped || !received.tryPop(current)) + return nullptr; - if (received.tryPop(current)) - { - auto * new_position = const_cast(current.message.data()); - BufferBase::set(new_position, current.message.size(), 0); - allowed = false; - - return true; - } - - return false; + return std::make_shared(current.message.data(), current.message.size()); } -void ReadBufferFromNATSConsumer::onMsg(natsConnection *, natsSubscription *, natsMsg * msg, void * consumer) +void NATSConsumer::onMsg(natsConnection *, natsSubscription *, natsMsg * msg, void * consumer) { - auto * buffer = static_cast(consumer); + auto * nats_consumer = static_cast(consumer); const int msg_length = natsMsg_GetDataLength(msg); if (msg_length) { String message_received = std::string(natsMsg_GetData(msg), msg_length); String subject = natsMsg_GetSubject(msg); - if (buffer->row_delimiter != '\0') - message_received += buffer->row_delimiter; MessageData data = { .message = message_received, .subject = subject, }; - if (!buffer->received.push(std::move(data))) + if (!nats_consumer->received.push(std::move(data))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); - buffer->storage.startStreaming(); + nats_consumer->storage.startStreaming(); } natsMsg_Destroy(msg); diff --git a/src/Storages/NATS/ReadBufferFromNATSConsumer.h b/src/Storages/NATS/NATSConsumer.h similarity index 84% rename from src/Storages/NATS/ReadBufferFromNATSConsumer.h rename to src/Storages/NATS/NATSConsumer.h index 306c0aff3bf..a6f950329aa 100644 --- a/src/Storages/NATS/ReadBufferFromNATSConsumer.h +++ b/src/Storages/NATS/NATSConsumer.h @@ -16,16 +16,15 @@ class Logger; namespace DB { -class ReadBufferFromNATSConsumer : public ReadBuffer +class NATSConsumer { public: - ReadBufferFromNATSConsumer( + NATSConsumer( std::shared_ptr connection_, StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, Poco::Logger * log_, - char row_delimiter_, uint32_t queue_size_, const std::atomic & stopped_); @@ -44,13 +43,14 @@ public: bool queueEmpty() { return received.empty(); } size_t queueSize() { return received.size(); } - void allowNext() { allowed = true; } // Allow to read next message. auto getSubject() const { return current.subject; } -private: - bool nextImpl() override; + /// Return read buffer containing next available message + /// or nullptr if there are no messages to process. + ReadBufferPtr consume(); +private: static void onMsg(natsConnection * nc, natsSubscription * sub, natsMsg * msg, void * consumer); std::shared_ptr connection; @@ -58,8 +58,6 @@ private: std::vector subscriptions; std::vector subjects; Poco::Logger * log; - char row_delimiter; - bool allowed = true; const std::atomic & stopped; bool subscribed = false; diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index b5812bc3349..7006e5633a9 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -26,7 +26,7 @@ NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : natsLibuv_Read, natsLibuv_Write, natsLibuv_Detach); - natsOptions_SetIOBufSize(opts, INT_MAX); + natsOptions_SetIOBufSize(opts, DBMS_DEFAULT_BUFFER_SIZE); natsOptions_SetSendAsap(opts, true); } diff --git a/src/Storages/NATS/NATSProducer.cpp b/src/Storages/NATS/NATSProducer.cpp new file mode 100644 index 00000000000..ed97ea17093 --- /dev/null +++ b/src/Storages/NATS/NATSProducer.cpp @@ -0,0 +1,126 @@ +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +static const auto BATCH = 1000; +static const auto MAX_BUFFERED = 131072; + +namespace ErrorCodes +{ + extern const int CANNOT_CONNECT_NATS; + extern const int LOGICAL_ERROR; +} + +NATSProducer::NATSProducer( + const NATSConfiguration & configuration_, + const String & subject_, + std::atomic & shutdown_called_, + Poco::Logger * log_) + : connection(configuration_, log_) + , subject(subject_) + , shutdown_called(shutdown_called_) + , payloads(BATCH) + , log(log_) +{ +} + +void NATSProducer::initialize() +{ + if (!connection.connect()) + throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to NATS {}", connection.connectionInfoForLog()); +} + +void NATSProducer::finishImpl() +{ + connection.disconnect(); +} + + +void NATSProducer::produce(const String & message, size_t, const Columns &, size_t) +{ + if (!payloads.push(message)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); +} + +void NATSProducer::publish() +{ + uv_thread_t flush_thread; + + uv_thread_create(&flush_thread, publishThreadFunc, static_cast(this)); + + connection.getHandler().startLoop(); + uv_thread_join(&flush_thread); +} + +void NATSProducer::publishThreadFunc(void * arg) +{ + NATSProducer * producer = static_cast(arg); + String payload; + + natsStatus status; + while (!producer->payloads.empty()) + { + if (natsConnection_Buffered(producer->connection.getConnection()) > MAX_BUFFERED) + break; + bool pop_result = producer->payloads.pop(payload); + + if (!pop_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); + + status = natsConnection_Publish(producer->connection.getConnection(), producer->subject.c_str(), payload.c_str(), payload.size()); + + if (status != NATS_OK) + { + LOG_DEBUG(producer->log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", + natsStatus_GetText(status), nats_GetLastError(nullptr)); + if (!producer->payloads.push(std::move(payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); + break; + } + } + + nats_ReleaseThreadMemory(); +} + +void NATSProducer::stopProducingTask() +{ + payloads.finish(); +} + +void NATSProducer::producingTask() +{ + try + { + while ((!payloads.isFinishedAndEmpty() || natsConnection_Buffered(connection.getConnection()) != 0) && !shutdown_called.load()) + { + publish(); + + if (!connection.isConnected()) + connection.reconnect(); + + iterateEventLoop(); + } + } + catch (...) + { + tryLogCurrentException(log); + } + + LOG_DEBUG(log, "Producer on subject {} completed", subject); +} + + +void NATSProducer::iterateEventLoop() +{ + connection.getHandler().iterateLoop(); +} + +} diff --git a/src/Storages/NATS/NATSProducer.h b/src/Storages/NATS/NATSProducer.h new file mode 100644 index 00000000000..6a0d087ff64 --- /dev/null +++ b/src/Storages/NATS/NATSProducer.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class NATSProducer : public ConcurrentMessageProducer +{ +public: + NATSProducer( + const NATSConfiguration & configuration_, + const String & subject_, + std::atomic & shutdown_called_, + Poco::Logger * log_); + + void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override; + +private: + String getProducingTaskName() const override { return "NatsProducingTask"; } + + void initialize() override; + void stopProducingTask() override; + void finishImpl() override; + + void producingTask() override; + + void iterateEventLoop(); + void publish(); + + static void publishThreadFunc(void * arg); + + NATSConnectionManager connection; + const String subject; + + /* false: when shutdown is called + * true: in all other cases + */ + std::atomic & shutdown_called; + + /* payloads.queue: + * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once + */ + ConcurrentBoundedQueue payloads; + + Poco::Logger * log; +}; + +} diff --git a/src/Storages/NATS/NATSSettings.h b/src/Storages/NATS/NATSSettings.h index 9bf9b969387..b4003eef46d 100644 --- a/src/Storages/NATS/NATSSettings.h +++ b/src/Storages/NATS/NATSSettings.h @@ -26,6 +26,7 @@ class ASTStorage; M(String, nats_password, "", "NATS password", 0) \ 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) \ #define LIST_OF_NATS_SETTINGS(M) \ NATS_RELATED_SETTINGS(M) \ diff --git a/src/Storages/NATS/NATSSink.cpp b/src/Storages/NATS/NATSSink.cpp deleted file mode 100644 index 44cf51072e6..00000000000 --- a/src/Storages/NATS/NATSSink.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -NATSSink::NATSSink( - StorageNATS & storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_, - ProducerBufferPtr buffer_) - : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) - , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , context(context_) - , buffer(buffer_) -{ -} - - -void NATSSink::onStart() -{ - buffer->activateWriting(); - - auto format_settings = getFormatSettings(context); - format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - - format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, - [this](const Columns & /* columns */, size_t /* rows */) - { - buffer->countRow(); - }, - format_settings); -} - - -void NATSSink::consume(Chunk chunk) -{ - format->write(getHeader().cloneWithColumns(chunk.detachColumns())); -} - - -void NATSSink::onFinish() -{ - format->finalize(); - - if (buffer) - buffer->updateMaxWait(); -} - -} diff --git a/src/Storages/NATS/NATSSink.h b/src/Storages/NATS/NATSSink.h deleted file mode 100644 index d94575de0e7..00000000000 --- a/src/Storages/NATS/NATSSink.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class IOutputFormat; -using IOutputFormatPtr = std::shared_ptr; - -class NATSSink : public SinkToStorage -{ -public: - explicit NATSSink(StorageNATS & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_, ProducerBufferPtr buffer_); - - void onStart() override; - void consume(Chunk chunk) override; - void onFinish() override; - - String getName() const override { return "NATSSink"; } - -private: - StorageNATS & storage; - StorageMetadataPtr metadata_snapshot; - ContextPtr context; - ProducerBufferPtr buffer; - IOutputFormatPtr format; -}; -} diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index f5e5e4f8b91..793b69e7e46 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -3,7 +3,8 @@ #include #include #include -#include +#include +#include namespace DB { @@ -59,11 +60,10 @@ NATSSource::~NATSSource() { storage.decrementReader(); - if (!buffer) + if (!consumer) return; - buffer->allowNext(); - storage.pushReadBuffer(buffer); + storage.pushConsumer(consumer); } bool NATSSource::checkTimeLimit() const @@ -81,21 +81,22 @@ bool NATSSource::checkTimeLimit() const Chunk NATSSource::generate() { - if (!buffer) + if (!consumer) { auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); - buffer = storage.popReadBuffer(timeout); - buffer->subscribe(); + consumer = storage.popConsumer(timeout); + consumer->subscribe(); } - if (!buffer || is_finished) + if (!consumer || is_finished) return {}; is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + EmptyReadBuffer empty_buf; auto input_format - = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + = FormatFactory::instance().getInputFormat(storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); @@ -103,22 +104,22 @@ Chunk NATSSource::generate() while (true) { - if (buffer->eof()) + if (consumer->queueEmpty()) break; - auto new_rows = executor.execute(); + size_t new_rows = 0; + if (auto buf = consumer->consume()) + new_rows = executor.execute(*buf); if (new_rows) { - auto subject = buffer->getSubject(); + auto subject = consumer->getSubject(); virtual_columns[0]->insertMany(subject, new_rows); total_rows = total_rows + new_rows; } - buffer->allowNext(); - - if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->isConsumerStopped() || !checkTimeLimit()) + if (total_rows >= max_block_size || consumer->queueEmpty() || consumer->isConsumerStopped() || !checkTimeLimit()) break; } diff --git a/src/Storages/NATS/NATSSource.h b/src/Storages/NATS/NATSSource.h index e4e94d2347a..604a8a5366f 100644 --- a/src/Storages/NATS/NATSSource.h +++ b/src/Storages/NATS/NATSSource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -21,11 +21,11 @@ public: ~NATSSource() override; String getName() const override { return storage.getName(); } - ConsumerBufferPtr getBuffer() { return buffer; } + NATSConsumerPtr getConsumer() { return consumer; } Chunk generate() override; - bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + bool queueEmpty() const { return !consumer || consumer->queueEmpty(); } void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } @@ -42,7 +42,7 @@ private: const Block non_virtual_header; const Block virtual_header; - ConsumerBufferPtr buffer; + NATSConsumerPtr consumer; Poco::Timespan max_execution_time = 0; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 4a3ba973e67..72f1501771e 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -11,10 +11,10 @@ #include #include #include -#include #include #include -#include +#include +#include #include #include #include @@ -23,7 +23,6 @@ #include #include #include -#include #include #include @@ -56,9 +55,9 @@ StorageNATS::StorageNATS( , nats_settings(std::move(nats_settings_)) , subjects(parseList(getContext()->getMacros()->expand(nats_settings->nats_subjects), ',')) , format_name(getContext()->getMacros()->expand(nats_settings->nats_format)) - , row_delimiter(nats_settings->nats_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) + , max_rows_per_message(nats_settings->nats_max_rows_per_message) , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, num_consumers) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) @@ -238,11 +237,11 @@ void StorageNATS::connectionFunc() bool StorageNATS::initBuffers() { size_t num_initialized = 0; - for (auto & buffer : buffers) + for (auto & consumer : consumers) { try { - buffer->subscribe(); + consumer->subscribe(); ++num_initialized; } catch (...) @@ -253,10 +252,10 @@ bool StorageNATS::initBuffers() } startLoop(); - const bool are_buffers_initialized = num_initialized == num_created_consumers; - if (are_buffers_initialized) + const bool are_consumers_initialized = num_initialized == num_created_consumers; + if (are_consumers_initialized) consumers_ready.store(true); - return are_buffers_initialized; + return are_consumers_initialized; } @@ -381,18 +380,24 @@ SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & m if (!isSubjectInSubscriptions(subject)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Selected subject is not among engine subjects"); - return std::make_shared(*this, metadata_snapshot, local_context, createWriteBuffer(subject)); -} + auto producer = std::make_unique(configuration, subject, shutdown_called, log); + size_t max_rows = max_rows_per_message; + /// Need for backward compatibility. + if (format_name == "Avro" && local_context->getSettingsRef().output_format_avro_rows_in_file.changed) + max_rows = local_context->getSettingsRef().output_format_avro_rows_in_file.value; + return std::make_shared( + metadata_snapshot->getSampleBlockNonMaterialized(), getFormatName(), max_rows, std::move(producer), getName(), modified_context);} void StorageNATS::startup() { + (void) is_attach; for (size_t i = 0; i < num_consumers; ++i) { try { - auto buffer = createReadBuffer(); - pushReadBuffer(std::move(buffer)); + auto consumer = createConsumer(); + pushConsumer(std::move(consumer)); ++num_created_consumers; } catch (...) @@ -425,14 +430,14 @@ void StorageNATS::shutdown() { if (drop_table) { - for (auto & buffer : buffers) - buffer->unsubscribe(); + for (auto & consumer : consumers) + consumer->unsubscribe(); } connection->disconnect(); for (size_t i = 0; i < num_created_consumers; ++i) - popReadBuffer(); + popConsumer(); } catch (...) { @@ -440,23 +445,23 @@ void StorageNATS::shutdown() } } -void StorageNATS::pushReadBuffer(ConsumerBufferPtr buffer) +void StorageNATS::pushConsumer(NATSConsumerPtr consumer) { - std::lock_guard lock(buffers_mutex); - buffers.push_back(buffer); + std::lock_guard lock(consumers_mutex); + consumers.push_back(consumer); semaphore.set(); } -ConsumerBufferPtr StorageNATS::popReadBuffer() +NATSConsumerPtr StorageNATS::popConsumer() { - return popReadBuffer(std::chrono::milliseconds::zero()); + return popConsumer(std::chrono::milliseconds::zero()); } -ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) +NATSConsumerPtr StorageNATS::popConsumer(std::chrono::milliseconds timeout) { - // Wait for the first free buffer + // Wait for the first free consumer if (timeout == std::chrono::milliseconds::zero()) semaphore.wait(); else @@ -465,29 +470,21 @@ ConsumerBufferPtr StorageNATS::popReadBuffer(std::chrono::milliseconds timeout) return nullptr; } - // Take the first available buffer from the list - std::lock_guard lock(buffers_mutex); - auto buffer = buffers.back(); - buffers.pop_back(); + // Take the first available consumer from the list + std::lock_guard lock(consumers_mutex); + auto consumer = consumers.back(); + consumers.pop_back(); - return buffer; + return consumer; } -ConsumerBufferPtr StorageNATS::createReadBuffer() +NATSConsumerPtr StorageNATS::createConsumer() { - return std::make_shared( + return std::make_shared( connection, *this, subjects, nats_settings->nats_queue_group.changed ? nats_settings->nats_queue_group.value : getStorageID().getFullTableName(), - log, row_delimiter, queue_size, shutdown_called); -} - - -ProducerBufferPtr StorageNATS::createWriteBuffer(const std::string & subject) -{ - return std::make_shared( - configuration, getContext(), subject, shutdown_called, log, - row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); + log, queue_size, shutdown_called); } bool StorageNATS::isSubjectInSubscriptions(const std::string & subject) @@ -643,7 +640,7 @@ bool StorageNATS::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { - LOG_DEBUG(log, "Current queue size: {}", buffers[0]->queueSize()); + LOG_DEBUG(log, "Current queue size: {}", consumers[0]->queueSize()); auto source = std::make_shared(*this, storage_snapshot, nats_context, column_names, block_size); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 185b39250c8..e29755d4e28 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -14,6 +13,9 @@ namespace DB { +class NATSConsumer; +using NATSConsumerPtr = std::shared_ptr; + class StorageNATS final : public IStorage, WithContext { public: @@ -51,9 +53,12 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; - void pushReadBuffer(ConsumerBufferPtr buf); - ConsumerBufferPtr popReadBuffer(); - ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + /// We want to control the number of rows in a chunk inserted into NATS + bool prefersLargeBlocks() const override { return false; } + + void pushConsumer(NATSConsumerPtr buf); + NATSConsumerPtr popConsumer(); + NATSConsumerPtr popConsumer(std::chrono::milliseconds timeout); const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; @@ -69,9 +74,9 @@ private: std::vector subjects; const String format_name; - char row_delimiter; const String schema_name; size_t num_consumers; + size_t max_rows_per_message; Poco::Logger * log; @@ -80,11 +85,11 @@ private: size_t num_created_consumers = 0; Poco::Semaphore semaphore; - std::mutex buffers_mutex; - std::vector buffers; /// available buffers for NATS consumers + std::mutex consumers_mutex; + std::vector consumers; /// available NATS consumers /// maximum number of messages in NATS queue (x-max-length). Also used - /// to setup size of inner buffer for received messages + /// to setup size of inner consumer for received messages uint32_t queue_size; std::once_flag flag; /// remove exchange only once @@ -114,8 +119,7 @@ private: mutable bool drop_table = false; bool is_attach; - ConsumerBufferPtr createReadBuffer(); - ProducerBufferPtr createWriteBuffer(const std::string & subject); + NATSConsumerPtr createConsumer(); bool isSubjectInSubscriptions(const std::string & subject); diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.cpp b/src/Storages/NATS/WriteBufferToNATSProducer.cpp deleted file mode 100644 index af76247d903..00000000000 --- a/src/Storages/NATS/WriteBufferToNATSProducer.cpp +++ /dev/null @@ -1,183 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -static const auto BATCH = 1000; -static const auto MAX_BUFFERED = 131072; - -namespace ErrorCodes -{ - extern const int CANNOT_CONNECT_NATS; - extern const int LOGICAL_ERROR; -} - -WriteBufferToNATSProducer::WriteBufferToNATSProducer( - const NATSConfiguration & configuration_, - ContextPtr global_context, - const String & subject_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_) - : WriteBuffer(nullptr, 0) - , connection(configuration_, log_) - , subject(subject_) - , shutdown_called(shutdown_called_) - , payloads(BATCH) - , log(log_) - , delim(delimiter) - , max_rows(rows_per_message) - , chunk_size(chunk_size_) -{ - if (!connection.connect()) - throw Exception(ErrorCodes::CANNOT_CONNECT_NATS, "Cannot connect to NATS {}", connection.connectionInfoForLog()); - - writing_task = global_context->getSchedulePool().createTask("NATSWritingTask", [this] { writingFunc(); }); - writing_task->deactivate(); - - reinitializeChunks(); -} - - -WriteBufferToNATSProducer::~WriteBufferToNATSProducer() -{ - writing_task->deactivate(); - assert(rows == 0); -} - - -void WriteBufferToNATSProducer::countRow() -{ - if (++rows % max_rows == 0) - { - const std::string & last_chunk = chunks.back(); - size_t last_chunk_size = offset(); - - if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) - --last_chunk_size; - - std::string payload; - payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); - - for (auto i = chunks.begin(), end = --chunks.end(); i != end; ++i) - payload.append(*i); - - payload.append(last_chunk, 0, last_chunk_size); - - reinitializeChunks(); - - if (!payloads.push(payload)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); - } -} - -void WriteBufferToNATSProducer::publish() -{ - uv_thread_t flush_thread; - - uv_thread_create(&flush_thread, publishThreadFunc, static_cast(this)); - - connection.getHandler().startLoop(); - uv_thread_join(&flush_thread); -} - -void WriteBufferToNATSProducer::publishThreadFunc(void * arg) -{ - WriteBufferToNATSProducer * buffer = static_cast(arg); - String payload; - - natsStatus status; - while (!buffer->payloads.empty()) - { - if (natsConnection_Buffered(buffer->connection.getConnection()) > MAX_BUFFERED) - break; - bool pop_result = buffer->payloads.pop(payload); - - if (!pop_result) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); - status = natsConnection_PublishString(buffer->connection.getConnection(), buffer->subject.c_str(), payload.c_str()); - - if (status != NATS_OK) - { - LOG_DEBUG(buffer->log, "Something went wrong during publishing to NATS subject. Nats status text: {}. Last error message: {}", - natsStatus_GetText(status), nats_GetLastError(nullptr)); - if (!buffer->payloads.push(std::move(payload))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); - break; - } - } - - nats_ReleaseThreadMemory(); -} - - -void WriteBufferToNATSProducer::writingFunc() -{ - try - { - while ((!payloads.empty() || wait_all) && !shutdown_called.load()) - { - publish(); - - LOG_DEBUG( - log, "Writing func {} {} {}", wait_payloads.load(), payloads.empty(), natsConnection_Buffered(connection.getConnection())); - if (wait_payloads.load() && payloads.empty() && natsConnection_Buffered(connection.getConnection()) == 0) - wait_all = false; - - if (!connection.isConnected() && wait_all) - connection.reconnect(); - - iterateEventLoop(); - } - } - catch (...) - { - tryLogCurrentException(log); - } - - LOG_DEBUG(log, "Producer on subject {} completed", subject); -} - - -void WriteBufferToNATSProducer::nextImpl() -{ - addChunk(); -} - -void WriteBufferToNATSProducer::addChunk() -{ - chunks.push_back(std::string()); - chunks.back().resize(chunk_size); - set(chunks.back().data(), chunk_size); -} - -void WriteBufferToNATSProducer::reinitializeChunks() -{ - rows = 0; - chunks.clear(); - /// We cannot leave the buffer in the undefined state (i.e. without any - /// underlying buffer), since in this case the WriteBuffeR::next() will - /// not call our nextImpl() (due to available() == 0) - addChunk(); -} - - -void WriteBufferToNATSProducer::iterateEventLoop() -{ - connection.getHandler().iterateLoop(); -} - -} diff --git a/src/Storages/NATS/WriteBufferToNATSProducer.h b/src/Storages/NATS/WriteBufferToNATSProducer.h deleted file mode 100644 index 484d80598db..00000000000 --- a/src/Storages/NATS/WriteBufferToNATSProducer.h +++ /dev/null @@ -1,81 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class WriteBufferToNATSProducer : public WriteBuffer -{ -public: - WriteBufferToNATSProducer( - const NATSConfiguration & configuration_, - ContextPtr global_context, - const String & subject_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_); - - ~WriteBufferToNATSProducer() override; - - void countRow(); - void activateWriting() { writing_task->activateAndSchedule(); } - void updateMaxWait() { wait_payloads.store(true); } - -private: - void nextImpl() override; - void addChunk(); - void reinitializeChunks(); - - void iterateEventLoop(); - void writingFunc(); - void publish(); - - static void publishThreadFunc(void * arg); - - NATSConnectionManager connection; - const String subject; - - /* false: when shutdown is called - * true: in all other cases - */ - std::atomic & shutdown_called; - - BackgroundSchedulePool::TaskHolder writing_task; - - /* payloads.queue: - * - payloads are pushed to queue in countRow and popped by another thread in writingFunc, each payload gets into queue only once - */ - ConcurrentBoundedQueue payloads; - - /* false: message delivery successfully ended: publisher received confirm from server that all published - * 1) persistent messages were written to disk - * 2) non-persistent messages reached the queue - * true: continue to process deliveries and returned messages - */ - bool wait_all = true; - - /* false: until writeSuffix is called - * true: means payloads.queue will not grow anymore - */ - std::atomic wait_payloads = false; - - Poco::Logger * log; - const std::optional delim; - const size_t max_rows; - const size_t chunk_size; - size_t rows = 0; - std::list chunks; -}; - -} diff --git a/src/Storages/RabbitMQ/Buffer_fwd.h b/src/Storages/RabbitMQ/Buffer_fwd.h deleted file mode 100644 index 5be2c6fdf6a..00000000000 --- a/src/Storages/RabbitMQ/Buffer_fwd.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class ReadBufferFromRabbitMQConsumer; -using ConsumerBufferPtr = std::shared_ptr; - -class WriteBufferToRabbitMQProducer; -using ProducerBufferPtr = std::shared_ptr; - -} diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp similarity index 78% rename from src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp rename to src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 3543085f5a0..c2a922b6baf 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -1,12 +1,11 @@ #include #include #include -#include #include #include -#include +#include #include -#include +#include #include #include "Poco/Timer.h" #include @@ -19,41 +18,33 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( +RabbitMQConsumer::RabbitMQConsumer( RabbitMQHandler & event_handler_, std::vector & queues_, size_t channel_id_base_, const String & channel_base_, Poco::Logger * log_, - char row_delimiter_, uint32_t queue_size_, const std::atomic & stopped_) - : ReadBuffer(nullptr, 0) - , event_handler(event_handler_) + : event_handler(event_handler_) , queues(queues_) , channel_base(channel_base_) , channel_id_base(channel_id_base_) , log(log_) - , row_delimiter(row_delimiter_) , stopped(stopped_) , received(queue_size_) { } -ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() -{ - BufferBase::set(nullptr, 0, 0); -} - -void ReadBufferFromRabbitMQConsumer::closeChannel() +void RabbitMQConsumer::closeChannel() { if (consumer_channel) consumer_channel->close(); } -void ReadBufferFromRabbitMQConsumer::subscribe() +void RabbitMQConsumer::subscribe() { for (const auto & queue_name : queues) { @@ -70,8 +61,6 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (message.bodySize()) { String message_received = std::string(message.body(), message.body() + message.bodySize()); - if (row_delimiter != '\0') - message_received += row_delimiter; if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", message.hasTimestamp() ? message.timestamp() : 0, @@ -91,7 +80,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe() } -bool ReadBufferFromRabbitMQConsumer::ackMessages() +bool RabbitMQConsumer::ackMessages() { AckTracker record_info = last_inserted_record_info; @@ -116,7 +105,7 @@ bool ReadBufferFromRabbitMQConsumer::ackMessages() } -void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record_info) +void RabbitMQConsumer::updateAckTracker(AckTracker record_info) { if (record_info.delivery_tag && channel_error.load()) return; @@ -128,7 +117,7 @@ void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record_info) } -void ReadBufferFromRabbitMQConsumer::setupChannel() +void RabbitMQConsumer::setupChannel() { if (!consumer_channel) return; @@ -159,7 +148,7 @@ void ReadBufferFromRabbitMQConsumer::setupChannel() } -bool ReadBufferFromRabbitMQConsumer::needChannelUpdate() +bool RabbitMQConsumer::needChannelUpdate() { if (wait_subscription) return false; @@ -168,27 +157,17 @@ bool ReadBufferFromRabbitMQConsumer::needChannelUpdate() } -void ReadBufferFromRabbitMQConsumer::iterateEventLoop() +void RabbitMQConsumer::iterateEventLoop() { event_handler.iterateLoop(); } - -bool ReadBufferFromRabbitMQConsumer::nextImpl() +ReadBufferPtr RabbitMQConsumer::consume() { - if (stopped || !allowed) - return false; + if (stopped || !received.tryPop(current)) + return nullptr; - if (received.tryPop(current)) - { - auto * new_position = const_cast(current.message.data()); - BufferBase::set(new_position, current.message.size(), 0); - allowed = false; - - return true; - } - - return false; + return std::make_shared(current.message.data(), current.message.size()); } } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/RabbitMQConsumer.h similarity index 88% rename from src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h rename to src/Storages/RabbitMQ/RabbitMQConsumer.h index bd55d169744..ca3daa5e090 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.h @@ -21,22 +21,19 @@ namespace DB class RabbitMQHandler; using ChannelPtr = std::unique_ptr; -class ReadBufferFromRabbitMQConsumer : public ReadBuffer +class RabbitMQConsumer { public: - ReadBufferFromRabbitMQConsumer( + RabbitMQConsumer( RabbitMQHandler & event_handler_, std::vector & queues_, size_t channel_id_base_, const String & channel_base_, Poco::Logger * log_, - char row_delimiter_, uint32_t queue_size_, const std::atomic & stopped_); - ~ReadBufferFromRabbitMQConsumer() override; - struct AckTracker { UInt64 delivery_tag; @@ -55,6 +52,10 @@ public: AckTracker track{}; }; + /// Return read buffer containing next available message + /// or nullptr if there are no messages to process. + ReadBufferPtr consume(); + ChannelPtr & getChannel() { return consumer_channel; } void setupChannel(); bool needChannelUpdate(); @@ -68,7 +69,6 @@ public: void updateAckTracker(AckTracker record = AckTracker()); bool queueEmpty() { return received.empty(); } - void allowNext() { allowed = true; } // Allow to read next message. auto getChannelID() const { return current.track.channel_id; } auto getDeliveryTag() const { return current.track.delivery_tag; } @@ -77,8 +77,6 @@ public: auto getTimestamp() const { return current.timestamp; } private: - bool nextImpl() override; - void subscribe(); void iterateEventLoop(); @@ -88,8 +86,6 @@ private: const String channel_base; const size_t channel_id_base; Poco::Logger * log; - char row_delimiter; - bool allowed = true; const std::atomic & stopped; String channel_id; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/RabbitMQProducer.cpp similarity index 67% rename from src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp rename to src/Storages/RabbitMQ/RabbitMQProducer.cpp index 367f8579a17..358c2b23d82 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQProducer.cpp @@ -1,12 +1,9 @@ -#include +#include #include #include -#include -#include #include #include -#include #include #include #include @@ -25,42 +22,35 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( - const RabbitMQConfiguration & configuration_, - ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_) - : WriteBuffer(nullptr, 0) - , connection(configuration_, log_) - , routing_keys(routing_keys_) - , exchange_name(exchange_name_) - , exchange_type(exchange_type_) - , channel_id_base(std::to_string(channel_id_base_)) - , persistent(persistent_) - , shutdown_called(shutdown_called_) - , payloads(BATCH) - , returned(RETURNED_LIMIT) - , log(log_) - , delim(delimiter) - , max_rows(rows_per_message) - , chunk_size(chunk_size_) +RabbitMQProducer::RabbitMQProducer( + const RabbitMQConfiguration & configuration_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_) + : connection(configuration_, log_) + , routing_keys(routing_keys_) + , exchange_name(exchange_name_) + , exchange_type(exchange_type_) + , channel_id_base(std::to_string(channel_id_base_)) + , persistent(persistent_) + , shutdown_called(shutdown_called_) + , payloads(BATCH) + , returned(RETURNED_LIMIT) + , log(log_) +{ +} + +void RabbitMQProducer::initialize() { if (connection.connect()) setupChannel(); else throw Exception(ErrorCodes::CANNOT_CONNECT_RABBITMQ, "Cannot connect to RabbitMQ {}", connection.connectionInfoForLog()); - writing_task = global_context->getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); - if (exchange_type == AMQP::ExchangeType::headers) { for (const auto & header : routing_keys) @@ -70,47 +60,32 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( key_arguments[matching[0]] = matching[1]; } } - - reinitializeChunks(); } - -WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() +void RabbitMQProducer::stopProducingTask() +{ + payloads.finish(); +} + +void RabbitMQProducer::finishImpl() { - writing_task->deactivate(); connection.disconnect(); - assert(rows == 0); } -void WriteBufferToRabbitMQProducer::countRow() +void RabbitMQProducer::produce(const String & message, size_t, const Columns &, size_t) { - if (++rows % max_rows == 0) - { - const std::string & last_chunk = chunks.back(); - size_t last_chunk_size = offset(); - - if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) - --last_chunk_size; - - std::string payload; - payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); - - for (auto i = chunks.begin(), end = --chunks.end(); i != end; ++i) - payload.append(*i); - - payload.append(last_chunk, 0, last_chunk_size); - - reinitializeChunks(); - - ++payload_counter; - if (!payloads.push(std::make_pair(payload_counter, payload))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); - } + LOG_DEBUG(&Poco::Logger::get("RabbitMQProducer"), "push {}", message); + + Payload payload; + payload.message = message; + payload.id = ++payload_counter; + if (!payloads.push(std::move(payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); } -void WriteBufferToRabbitMQProducer::setupChannel() +void RabbitMQProducer::setupChannel() { producer_channel = connection.createChannel(); @@ -162,7 +137,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() } -void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) +void RabbitMQProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) { auto record_iter = delivery_record.find(received_delivery_tag); assert(record_iter != delivery_record.end()); @@ -191,9 +166,9 @@ void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, b } -void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) +void RabbitMQProducer::publish(Payloads & messages, bool republishing) { - std::pair payload; + Payload payload; /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged * messages cannot exceed returned.size(), because they all might end up there @@ -203,9 +178,11 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable() will anyway return true, /// but must publish only after onReady callback. @@ -273,43 +252,20 @@ void WriteBufferToRabbitMQProducer::writingFunc() iterateEventLoop(); - if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) - wait_all = false; - else if (!producer_channel->usable()) + if (!producer_channel->usable()) { if (connection.reconnect()) setupChannel(); } } + LOG_DEBUG(&Poco::Logger::get("RabbitMQProducer"), "finish producingTask"); + LOG_DEBUG(log, "Producer on channel {} completed", channel_id); } -void WriteBufferToRabbitMQProducer::nextImpl() -{ - addChunk(); -} - -void WriteBufferToRabbitMQProducer::addChunk() -{ - chunks.push_back(std::string()); - chunks.back().resize(chunk_size); - set(chunks.back().data(), chunk_size); -} - -void WriteBufferToRabbitMQProducer::reinitializeChunks() -{ - rows = 0; - chunks.clear(); - /// We cannot leave the buffer in the undefined state (i.e. without any - /// underlying buffer), since in this case the WriteBuffeR::next() will - /// not call our nextImpl() (due to available() == 0) - addChunk(); -} - - -void WriteBufferToRabbitMQProducer::iterateEventLoop() +void RabbitMQProducer::iterateEventLoop() { connection.getHandler().iterateLoop(); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/RabbitMQProducer.h similarity index 64% rename from src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h rename to src/Storages/RabbitMQ/RabbitMQProducer.h index ee6720ece13..6e3f43f5f4b 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/RabbitMQProducer.h @@ -1,53 +1,53 @@ #pragma once -#include #include #include #include #include #include #include +#include #include -#include #include namespace DB { -class WriteBufferToRabbitMQProducer : public WriteBuffer +class RabbitMQProducer : public ConcurrentMessageProducer { public: - WriteBufferToRabbitMQProducer( - const RabbitMQConfiguration & configuration_, - ContextPtr global_context, - const Names & routing_keys_, - const String & exchange_name_, - const AMQP::ExchangeType exchange_type_, - const size_t channel_id_base_, - const bool persistent_, - std::atomic & shutdown_called_, - Poco::Logger * log_, - std::optional delimiter, - size_t rows_per_message, - size_t chunk_size_ - ); + RabbitMQProducer( + const RabbitMQConfiguration & configuration_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, + const size_t channel_id_base_, + const bool persistent_, + std::atomic & shutdown_called_, + Poco::Logger * log_); - ~WriteBufferToRabbitMQProducer() override; - - void countRow(); - void activateWriting() { writing_task->activateAndSchedule(); } - void updateMaxWait() { wait_num.store(payload_counter); } + void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override; private: - void nextImpl() override; - void addChunk(); - void reinitializeChunks(); + String getProducingTaskName() const override { return "RabbitMQProducingTask"; } + + struct Payload + { + String message; + UInt64 id; + }; + + using Payloads = ConcurrentBoundedQueue; + + void initialize() override; + void stopProducingTask() override; + void finishImpl() override; void iterateEventLoop(); - void writingFunc(); + void producingTask() override; void setupChannel(); void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); - void publish(ConcurrentBoundedQueue> & message, bool republishing); + void publish(Payloads & messages, bool republishing); RabbitMQConnection connection; @@ -63,7 +63,6 @@ private: std::atomic & shutdown_called; AMQP::Table key_arguments; - BackgroundSchedulePool::TaskHolder writing_task; std::unique_ptr producer_channel; bool producer_ready = false; @@ -84,7 +83,7 @@ private: * to disk or it was unable to reach the queue. * - payloads are popped from the queue once republished */ - ConcurrentBoundedQueue> payloads, returned; + Payloads payloads, returned; /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was @@ -97,25 +96,15 @@ private: * 2) non-persistent messages reached the queue * true: continue to process deliveries and returned messages */ - bool wait_all = true; - - /* false: until writeSuffix is called - * true: means payloads.queue will not grow anymore - */ - std::atomic wait_num = 0; +// bool wait_all = true; /// Needed to fill messageID property UInt64 payload_counter = 0; /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue - std::map> delivery_record; + std::map delivery_record; Poco::Logger * log; - const std::optional delim; - const size_t max_rows; - const size_t chunk_size; - size_t rows = 0; - std::list chunks; }; } diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 38baeadad2a..739230d25f5 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -31,6 +31,7 @@ namespace DB M(String, rabbitmq_username, "", "RabbitMQ username", 0) \ M(String, rabbitmq_password, "", "RabbitMQ password", 0) \ M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \ + M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \ #define LIST_OF_RABBITMQ_SETTINGS(M) \ RABBITMQ_RELATED_SETTINGS(M) \ diff --git a/src/Storages/RabbitMQ/RabbitMQSink.cpp b/src/Storages/RabbitMQ/RabbitMQSink.cpp deleted file mode 100644 index 5254966af4c..00000000000 --- a/src/Storages/RabbitMQ/RabbitMQSink.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -RabbitMQSink::RabbitMQSink( - StorageRabbitMQ & storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_) - : SinkToStorage(metadata_snapshot_->getSampleBlockNonMaterialized()) - , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , context(context_) -{ - storage.unbindExchange(); -} - - -void RabbitMQSink::onStart() -{ - buffer = storage.createWriteBuffer(); - buffer->activateWriting(); - - auto format_settings = getFormatSettings(context); - format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - - format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, - [this](const Columns & /* columns */, size_t /* rows */) - { - buffer->countRow(); - }, - format_settings); -} - - -void RabbitMQSink::consume(Chunk chunk) -{ - format->write(getHeader().cloneWithColumns(chunk.detachColumns())); -} - - -void RabbitMQSink::onFinish() -{ - format->finalize(); - - if (buffer) - buffer->updateMaxWait(); -} - -} diff --git a/src/Storages/RabbitMQ/RabbitMQSink.h b/src/Storages/RabbitMQ/RabbitMQSink.h deleted file mode 100644 index 02014a3f89e..00000000000 --- a/src/Storages/RabbitMQ/RabbitMQSink.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class IOutputFormat; -using IOutputFormatPtr = std::shared_ptr; - -class RabbitMQSink : public SinkToStorage -{ - -public: - explicit RabbitMQSink(StorageRabbitMQ & storage_, const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_); - - void onStart() override; - void consume(Chunk chunk) override; - void onFinish() override; - - String getName() const override { return "RabbitMQSink"; } - -private: - StorageRabbitMQ & storage; - StorageMetadataPtr metadata_snapshot; - ContextPtr context; - ProducerBufferPtr buffer; - IOutputFormatPtr format; -}; -} diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index f6e5bb84037..98c8bfa9189 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -3,7 +3,8 @@ #include #include #include -#include +#include +#include namespace DB { @@ -70,31 +71,31 @@ RabbitMQSource::~RabbitMQSource() { storage.decrementReader(); - if (!buffer) + if (!consumer) return; - storage.pushReadBuffer(buffer); + storage.pushConsumer(consumer); } bool RabbitMQSource::needChannelUpdate() { - if (!buffer) + if (!consumer) return false; - return buffer->needChannelUpdate(); + return consumer->needChannelUpdate(); } void RabbitMQSource::updateChannel() { - if (!buffer) + if (!consumer) return; - buffer->updateAckTracker(); + consumer->updateAckTracker(); - if (storage.updateChannel(buffer->getChannel())) - buffer->setupChannel(); + if (storage.updateChannel(consumer->getChannel())) + consumer->setupChannel(); } Chunk RabbitMQSource::generate() @@ -121,20 +122,21 @@ bool RabbitMQSource::checkTimeLimit() const Chunk RabbitMQSource::generateImpl() { - if (!buffer) + if (!consumer) { auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); - buffer = storage.popReadBuffer(timeout); + consumer = storage.popConsumer(timeout); } - if (!buffer || is_finished) + if (!consumer || is_finished) return {}; is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); @@ -142,21 +144,23 @@ Chunk RabbitMQSource::generateImpl() while (true) { - if (buffer->queueEmpty()) + if (consumer->queueEmpty()) break; - auto new_rows = executor.execute(); + size_t new_rows = 0; + if (auto buf = consumer->consume()) + new_rows = executor.execute(*buf); if (new_rows) { auto exchange_name = storage.getExchange(); - auto channel_id = buffer->getChannelID(); - auto delivery_tag = buffer->getDeliveryTag(); - auto redelivered = buffer->getRedelivered(); - auto message_id = buffer->getMessageID(); - auto timestamp = buffer->getTimestamp(); + auto channel_id = consumer->getChannelID(); + auto delivery_tag = consumer->getDeliveryTag(); + auto redelivered = consumer->getRedelivered(); + auto message_id = consumer->getMessageID(); + auto timestamp = consumer->getTimestamp(); - buffer->updateAckTracker({delivery_tag, channel_id}); + consumer->updateAckTracker({delivery_tag, channel_id}); for (size_t i = 0; i < new_rows; ++i) { @@ -171,9 +175,7 @@ Chunk RabbitMQSource::generateImpl() total_rows = total_rows + new_rows; } - buffer->allowNext(); - - if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->isConsumerStopped() || !checkTimeLimit()) + if (total_rows >= max_block_size || consumer->queueEmpty() || consumer->isConsumerStopped() || !checkTimeLimit()) break; } @@ -190,10 +192,10 @@ Chunk RabbitMQSource::generateImpl() bool RabbitMQSource::sendAck() { - if (!buffer) + if (!consumer) return false; - if (!buffer->ackMessages()) + if (!consumer->ackMessages()) return false; return true; diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index bd2882d1938..6d948edfb59 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -23,11 +23,11 @@ public: ~RabbitMQSource() override; String getName() const override { return storage.getName(); } - ConsumerBufferPtr getBuffer() { return buffer; } + RabbitMQConsumerPtr getBuffer() { return consumer; } Chunk generate() override; - bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + bool queueEmpty() const { return !consumer || consumer->queueEmpty(); } bool needChannelUpdate(); void updateChannel(); bool sendAck(); @@ -47,7 +47,7 @@ private: const Block non_virtual_header; const Block virtual_header; - ConsumerBufferPtr buffer; + RabbitMQConsumerPtr consumer; Poco::Timespan max_execution_time = 0; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 70838daec24..ec5d7329f8c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -13,11 +13,11 @@ #include #include #include +#include #include -#include #include #include -#include +#include #include #include #include @@ -77,12 +77,12 @@ StorageRabbitMQ::StorageRabbitMQ( , format_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_format)) , exchange_type(defineExchangeType(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_type))) , routing_keys(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_routing_key_list))) - , row_delimiter(rabbitmq_settings->rabbitmq_row_delimiter.value) , schema_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_schema)) , num_consumers(rabbitmq_settings->rabbitmq_num_consumers.value) , num_queues(rabbitmq_settings->rabbitmq_num_queues.value) , queue_base(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_base)) , queue_settings_list(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_settings_list))) + , max_rows_per_message(rabbitmq_settings->rabbitmq_max_rows_per_message) , persistent(rabbitmq_settings->rabbitmq_persistent.value) , use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value) , hash_exchange(num_consumers > 1 || num_queues > 1) @@ -607,18 +607,18 @@ bool StorageRabbitMQ::updateChannel(ChannelPtr & channel) } -void StorageRabbitMQ::prepareChannelForBuffer(ConsumerBufferPtr buffer) +void StorageRabbitMQ::prepareChannelForConsumer(RabbitMQConsumerPtr consumer) { - if (!buffer) + if (!consumer) return; - if (buffer->queuesCount() != queues.size()) - buffer->updateQueues(queues); + if (consumer->queuesCount() != queues.size()) + consumer->updateQueues(queues); - buffer->updateAckTracker(); + consumer->updateAckTracker(); - if (updateChannel(buffer->getChannel())) - buffer->setupChannel(); + if (updateChannel(consumer->getChannel())) + consumer->setupChannel(); } @@ -749,7 +749,19 @@ void StorageRabbitMQ::read( SinkToStoragePtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - return std::make_shared(*this, metadata_snapshot, local_context); + auto producer = std::make_unique( + configuration, routing_keys, exchange_name, exchange_type, producer_id.fetch_add(1), persistent, shutdown_called, log); + size_t max_rows = max_rows_per_message; + /// Need for backward compatibility. + if (format_name == "Avro" && local_context->getSettingsRef().output_format_avro_rows_in_file.changed) + max_rows = local_context->getSettingsRef().output_format_avro_rows_in_file.value; + return std::make_shared( + metadata_snapshot->getSampleBlockNonMaterialized(), + getFormatName(), + max_rows, + std::move(producer), + getName(), + local_context); } @@ -780,8 +792,8 @@ void StorageRabbitMQ::startup() { try { - auto buffer = createReadBuffer(); - pushReadBuffer(std::move(buffer)); + auto consumer = createConsumer(); + pushConsumer(std::move(consumer)); ++num_created_consumers; } catch (...) @@ -813,18 +825,18 @@ void StorageRabbitMQ::shutdown() { if (drop_table) { - for (auto & buffer : buffers) - buffer->closeChannel(); + for (auto & consumer : consumers) + consumer->closeChannel(); cleanupRabbitMQ(); } - /// It is important to close connection here - before removing consumer buffers, because - /// it will finish and clean callbacks, which might use those buffers data. + /// It is important to close connection here - before removing consumers, because + /// it will finish and clean callbacks, which might use those consumers data. connection->disconnect(); for (size_t i = 0; i < num_created_consumers; ++i) - popReadBuffer(); + popConsumer(); } catch (...) { @@ -884,23 +896,23 @@ void StorageRabbitMQ::cleanupRabbitMQ() const } -void StorageRabbitMQ::pushReadBuffer(ConsumerBufferPtr buffer) +void StorageRabbitMQ::pushConsumer(RabbitMQConsumerPtr consumer) { - std::lock_guard lock(buffers_mutex); - buffers.push_back(buffer); + std::lock_guard lock(consumers_mutex); + consumers.push_back(consumer); semaphore.set(); } -ConsumerBufferPtr StorageRabbitMQ::popReadBuffer() +RabbitMQConsumerPtr StorageRabbitMQ::popConsumer() { - return popReadBuffer(std::chrono::milliseconds::zero()); + return popConsumer(std::chrono::milliseconds::zero()); } -ConsumerBufferPtr StorageRabbitMQ::popReadBuffer(std::chrono::milliseconds timeout) +RabbitMQConsumerPtr StorageRabbitMQ::popConsumer(std::chrono::milliseconds timeout) { - // Wait for the first free buffer + // Wait for the first free consumer if (timeout == std::chrono::milliseconds::zero()) semaphore.wait(); else @@ -909,32 +921,22 @@ ConsumerBufferPtr StorageRabbitMQ::popReadBuffer(std::chrono::milliseconds timeo return nullptr; } - // Take the first available buffer from the list - std::lock_guard lock(buffers_mutex); - auto buffer = buffers.back(); - buffers.pop_back(); + // Take the first available consumer from the list + std::lock_guard lock(consumers_mutex); + auto consumer = consumers.back(); + consumers.pop_back(); - return buffer; + return consumer; } -ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() +RabbitMQConsumerPtr StorageRabbitMQ::createConsumer() { - return std::make_shared( + return std::make_shared( connection->getHandler(), queues, ++consumer_id, - unique_strbase, log, row_delimiter, queue_size, shutdown_called); + unique_strbase, log, queue_size, shutdown_called); } - -ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() -{ - return std::make_shared( - configuration, getContext(), routing_keys, exchange_name, exchange_type, - producer_id.fetch_add(1), persistent, shutdown_called, log, - row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); -} - - bool StorageRabbitMQ::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached @@ -968,8 +970,8 @@ void StorageRabbitMQ::initializeBuffers() assert(rabbit_is_ready); if (!initialized) { - for (const auto & buffer : buffers) - prepareChannelForBuffer(buffer); + for (const auto & consumer : consumers) + prepareChannelForConsumer(consumer); initialized = true; } } @@ -1128,8 +1130,8 @@ bool StorageRabbitMQ::streamToViews() if (source->needChannelUpdate()) { - auto buffer = source->getBuffer(); - prepareChannelForBuffer(buffer); + auto consumer = source->getBuffer(); + prepareChannelForConsumer(consumer); } /* false is returned by the sendAck function in only two cases: diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 455b2fe8f09..b1ca7277be9 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -17,6 +17,8 @@ namespace DB { +using RabbitMQConsumerPtr = std::shared_ptr; + class StorageRabbitMQ final: public IStorage, WithContext { public: @@ -57,11 +59,12 @@ public: const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; - void pushReadBuffer(ConsumerBufferPtr buf); - ConsumerBufferPtr popReadBuffer(); - ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); + /// We want to control the number of rows in a chunk inserted into RabbitMQ + bool prefersLargeBlocks() const override { return false; } - ProducerBufferPtr createWriteBuffer(); + void pushConsumer(RabbitMQConsumerPtr buf); + RabbitMQConsumerPtr popConsumer(); + RabbitMQConsumerPtr popConsumer(std::chrono::milliseconds timeout); const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; @@ -71,7 +74,7 @@ public: bool updateChannel(ChannelPtr & channel); void updateQueues(std::vector & queues_) { queues_ = queues; } - void prepareChannelForBuffer(ConsumerBufferPtr buffer); + void prepareChannelForConsumer(RabbitMQConsumerPtr consumer); void incrementReader(); void decrementReader(); @@ -84,12 +87,12 @@ private: const String format_name; AMQP::ExchangeType exchange_type; Names routing_keys; - char row_delimiter; const String schema_name; size_t num_consumers; size_t num_queues; String queue_base; Names queue_settings_list; + size_t max_rows_per_message; /// For insert query. Mark messages as durable. const bool persistent; @@ -107,17 +110,17 @@ private: size_t num_created_consumers = 0; Poco::Semaphore semaphore; - std::mutex buffers_mutex; - std::vector buffers; /// available buffers for RabbitMQ consumers + std::mutex consumers_mutex; + std::vector consumers; /// available RabbitMQ consumers String unique_strbase; /// to make unique consumer channel id /// maximum number of messages in RabbitMQ queue (x-max-length). Also used - /// to setup size of inner buffer for received messages + /// to setup size of inner consumer for received messages uint32_t queue_size; String sharding_exchange, bridge_exchange, consumer_exchange; - size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id + size_t consumer_id = 0; /// counter for consumer, needed for channel id std::vector queues; @@ -135,8 +138,8 @@ private: /// Needed for tell MV or producer background tasks /// that they must finish as soon as possible. std::atomic shutdown_called{false}; - /// Counter for producer buffers, needed for channel id. - /// Needed to generate unique producer buffer identifiers. + /// Counter for producers, needed for channel id. + /// Needed to generate unique producer identifiers. std::atomic producer_id = 1; /// Has connection background task completed successfully? /// It is started only once -- in constructor. @@ -160,7 +163,7 @@ private: mutable bool drop_table = false; bool is_attach; - ConsumerBufferPtr createReadBuffer(); + RabbitMQConsumerPtr createConsumer(); void initializeBuffers(); bool initialized = false; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 29f2d0667d9..f9512c3b4ea 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -820,8 +820,7 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name, - *write_buf, metadata_snapshot->getSampleBlock(), context, - {}, format_settings); + *write_buf, metadata_snapshot->getSampleBlock(), context, format_settings); if (do_not_write_prefix) writer->doNotWritePrefix(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 675dd548088..27bb6cf34d0 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -622,7 +622,7 @@ public: compression_method, 3); writer - = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); + = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); } String getName() const override { return "StorageS3Sink"; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f1c924a3448..d0d7a950415 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -439,7 +439,7 @@ StorageURLSink::StorageURLSink( std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, timeouts), compression_method, 3); - writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); + writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings); } diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 78d3debd4e0..a147878da9b 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -450,6 +450,7 @@ "formatReadableSize" "formatReadableTimeDelta" "formatRow" +"formatRowNoNewline" "FQDN" "fragment" "FREEZE" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 1b4d6dcd4c5..5fdeef88aa9 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -176,6 +176,7 @@ "tan" "substring" "formatRow" +"formatRowNoNewline" "dictGetHierarchy" "notEmpty" "format" diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 46bf7b0b3a0..c4ad2eb4080 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -3528,7 +3528,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n': Could not print diagnostic info because two last rows aren't in buffer (rare case)\\n"}""", + "expected": """{"raw_message":"0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", "printable": True, "supports_empty_value": True, }, @@ -3540,7 +3540,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n': Could not print diagnostic info because two last rows aren't in buffer (rare case)\\n"}""", + "expected": """{"raw_message":"0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", "supports_empty_value": True, "printable": True, }, @@ -3552,7 +3552,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"\\"id\\",\\"blockNo\\",\\"val1\\",\\"val2\\",\\"val3\\"\\n0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n': Could not print diagnostic info because two last rows aren't in buffer (rare case)\\n"}""", + "expected": """{"raw_message":"\\"id\\",\\"blockNo\\",\\"val1\\",\\"val2\\",\\"val3\\"\\n0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", "printable": True, }, "Values": { @@ -3575,7 +3575,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n': Could not print diagnostic info because two last rows aren't in buffer (rare case)\\n"}""", + "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n"}""", "supports_empty_value": True, "printable": True, }, @@ -3587,7 +3587,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\nInt64\\tUInt16\\tString\\tFloat32\\tUInt8\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n': Could not print diagnostic info because two last rows aren't in buffer (rare case)\\n"}""", + "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\nInt64\\tUInt16\\tString\\tFloat32\\tUInt8\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", "printable": True, }, "Native": { @@ -4127,6 +4127,315 @@ def test_num_consumers_limit(kafka_cluster): instance.query("DROP TABLE test.kafka") +def test_format_with_prefix_and_suffix(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + kafka_create_topic(admin_client, "custom") + + instance.query( + """ + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'custom', + kafka_group_name = 'custom', + kafka_format = 'CustomSeparated'; + """ + ) + + instance.query( + "INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + messages = [] + + attempt = 0 + while attempt < 100: + messages.extend(kafka_consume(kafka_cluster, "custom")) + if len(messages) == 2: + break + attempt += 1 + + assert len(messages) == 2 + + assert ( + "".join(messages) == "\n0\t0\n\n\n10\t100\n\n" + ) + + kafka_delete_topic(admin_client, "custom") + + +def test_max_rows_per_message(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic = "custom_max_rows_per_message" + + kafka_create_topic(admin_client, topic) + + num_rows = 5 + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = 'CustomSeparated', + format_custom_result_before_delimiter = '\n', + format_custom_result_after_delimiter = '\n', + kafka_max_rows_per_message = 3; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + """ + ) + + instance.query( + f"INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + messages = [] + + attempt = 0 + while attempt < 100: + messages.extend(kafka_consume(kafka_cluster, topic)) + if len(messages) == 2: + break + attempt += 1 + + assert len(messages) == 2 + + assert ( + "".join(messages) + == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" + ) + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view") + assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" + + kafka_delete_topic(admin_client, topic) + + +def test_row_based_formats(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + for format_name in [ + "TSV", + "TSVWithNamesAndTypes", + "TSKV", + "CSV", + "CSVWithNamesAndTypes", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "Avro", + "RowBinary", + "RowBinaryWithNamesAndTypes", + "MsgPack", + ]: + + print(format_name) + + kafka_create_topic(admin_client, format_name) + + num_rows = 10 + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}', + kafka_max_rows_per_message = 5; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + """ + ) + + messages = [] + + attempt = 0 + while attempt < 100: + messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) + if len(messages) == 2: + break + attempt += 1 + + assert len(messages) == 2 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected + + kafka_delete_topic(admin_client, format_name) + + +def test_block_based_formats_1(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic = "pretty_space" + kafka_create_topic(admin_client, topic) + + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = 'PrettySpace'; + + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0; + """ + ) + + messages = [] + + attempt = 0 + while attempt < 100: + messages.extend(kafka_consume(kafka_cluster, topic)) + if len(messages) == 3: + break + attempt += 1 + + assert len(messages) == 3 + + data = [] + for message in messages: + splitted = message.split("\n") + assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" + assert splitted[1] == "" + assert splitted[-1] == "" + data += [line.split() for line in splitted[2:-1]] + + assert data == [ + ["0", "0"], + ["10", "100"], + ["20", "200"], + ["30", "300"], + ["40", "400"], + ] + + kafka_delete_topic(admin_client, topic) + + +def test_block_based_formats_2(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + num_rows = 100 + + for format_name in [ + "JSONColumns", + "Native", + "Arrow", + "Parquet", + "ORC", + "JSONCompactColumns", + ]: + + kafka_create_topic(admin_client, format_name) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; + """ + ) + + messages = [] + + attempt = 0 + while attempt < 100: + messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) + if len(messages) == 9: + break + attempt += 1 + + assert len(messages) == 9 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view ORDER by key") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected + + kafka_delete_topic(admin_client, format_name) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index 63dde8922a6..209a5dd1a9b 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -1492,6 +1492,376 @@ def test_nats_predefined_configuration(nats_cluster): break +def test_format_with_prefix_and_suffix(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.nats; + + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'custom', + nats_format = 'CustomSeparated'; + """ + ) + while not check_table_is_ready(instance, "test.nats"): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + insert_messages = [] + + async def sub_to_nats(): + nc = await nats_connect_ssl( + nats_cluster.nats_port, + user="click", + password="house", + ssl_ctx=nats_cluster.nats_ssl_context, + ) + sub = await nc.subscribe("custom") + await sub.unsubscribe(2) + async for msg in sub.messages: + insert_messages.append(msg.data.decode()) + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + instance.query( + "INSERT INTO test.nats select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + thread.join() + + assert ( + "".join(insert_messages) + == "\n0\t0\n\n\n10\t100\n\n" + ) + + +def test_max_rows_per_message(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.nats; + + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'custom1', + nats_format = 'CustomSeparated', + nats_max_rows_per_message = 3, + format_custom_result_before_delimiter = '\n', + format_custom_result_after_delimiter = '\n'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.nats; + """ + ) + while not check_table_is_ready(instance, "test.nats"): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + num_rows = 5 + + insert_messages = [] + + async def sub_to_nats(): + nc = await nats_connect_ssl( + nats_cluster.nats_port, + user="click", + password="house", + ssl_ctx=nats_cluster.nats_ssl_context, + ) + sub = await nc.subscribe("custom1") + await sub.unsubscribe(2) + async for msg in sub.messages: + insert_messages.append(msg.data.decode()) + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + instance.query( + f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + thread.join() + + assert ( + "".join(insert_messages) + == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" + ) + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view") + assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" + + +def test_row_based_formats(nats_cluster): + num_rows = 10 + + for format_name in [ + "TSV", + "TSVWithNamesAndTypes", + "TSKV", + "CSV", + "CSVWithNamesAndTypes", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "Avro", + "RowBinary", + "RowBinaryWithNamesAndTypes", + "MsgPack", + ]: + print(format_name) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.nats; + + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = '{format_name}', + nats_format = '{format_name}'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.nats; + """ + ) + + while not check_table_is_ready(instance, "test.nats"): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + insert_messages = 0 + + async def sub_to_nats(): + nc = await nats_connect_ssl( + nats_cluster.nats_port, + user="click", + password="house", + ssl_ctx=nats_cluster.nats_ssl_context, + ) + sub = await nc.subscribe(format_name) + await sub.unsubscribe(2) + async for msg in sub.messages: + nonlocal insert_messages + insert_messages += 1 + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + instance.query( + f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows})" + ) + + thread.join() + + assert insert_messages == 2 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + + result = instance.query("SELECT * FROM test.view") + assert result == expected + + +def test_block_based_formats_1(nats_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.nats; + + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = 'PrettySpace', + nats_format = 'PrettySpace'; + """ + ) + + insert_messages = [] + + async def sub_to_nats(): + nc = await nats_connect_ssl( + nats_cluster.nats_port, + user="click", + password="house", + ssl_ctx=nats_cluster.nats_ssl_context, + ) + sub = await nc.subscribe("PrettySpace") + await sub.unsubscribe(3) + async for msg in sub.messages: + insert_messages.append(msg.data.decode()) + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + attempt = 0 + while attempt < 100: + try: + instance.query( + "INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0;" + ) + break + except Exception: + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + attempt += 1 + thread.join() + + data = [] + for message in insert_messages: + splitted = message.split("\n") + assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" + assert splitted[1] == "" + assert splitted[-1] == "" + data += [line.split() for line in splitted[2:-1]] + + assert data == [ + ["0", "0"], + ["10", "100"], + ["20", "200"], + ["30", "300"], + ["40", "400"], + ] + + +def test_block_based_formats_2(nats_cluster): + num_rows = 100 + + for format_name in [ + "JSONColumns", + "Native", + "Arrow", + "Parquet", + "ORC", + "JSONCompactColumns", + ]: + print(format_name) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.nats; + + CREATE TABLE test.nats (key UInt64, value UInt64) + ENGINE = NATS + SETTINGS nats_url = 'nats1:4444', + nats_subjects = '{format_name}', + nats_format = '{format_name}'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.nats; + """ + ) + + while not check_table_is_ready(instance, "test.nats"): + logging.debug("Table test.nats is not yet ready") + time.sleep(0.5) + + insert_messages = 0 + + async def sub_to_nats(): + nc = await nats_connect_ssl( + nats_cluster.nats_port, + user="click", + password="house", + ssl_ctx=nats_cluster.nats_ssl_context, + ) + sub = await nc.subscribe(format_name) + await sub.unsubscribe(9) + async for msg in sub.messages: + nonlocal insert_messages + insert_messages += 1 + + await sub.drain() + await nc.drain() + + def run_sub(): + asyncio.run(sub_to_nats()) + + thread = threading.Thread(target=run_sub) + thread.start() + time.sleep(1) + + instance.query( + f"INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;" + ) + + thread.join() + + assert insert_messages == 9 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view ORDER by key") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index f5d216a8b92..63b8d1215aa 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2848,3 +2848,355 @@ def test_rabbitmq_address(rabbitmq_cluster): instance2.query("drop table rabbit_in sync") instance2.query("drop table rabbit_out sync") + + +def test_format_with_prefix_and_suffix(rabbitmq_cluster): + instance.query( + """ + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'insert', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'custom', + rabbitmq_format = 'CustomSeparated'; + """ + ) + + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials + ) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + result = consumer.queue_declare(queue="") + queue_name = result.method.queue + consumer.queue_bind(exchange="insert", queue=queue_name, routing_key="custom") + + instance.query( + "INSERT INTO test.rabbitmq select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + insert_messages = [] + + def onReceived(channel, method, properties, body): + insert_messages.append(body.decode()) + if len(insert_messages) == 2: + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + assert ( + "".join(insert_messages) + == "\n0\t0\n\n\n10\t100\n\n" + ) + + +def test_max_rows_per_message(rabbitmq_cluster): + num_rows = 5 + + instance.query( + """ + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbit; + + CREATE TABLE test.rabbit (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_format = 'CustomSeparated', + rabbitmq_exchange_name = 'custom', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'custom1', + rabbitmq_max_rows_per_message = 3, + format_custom_result_before_delimiter = '\n', + format_custom_result_after_delimiter = '\n'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.rabbit; + """ + ) + + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials + ) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + result = consumer.queue_declare(queue="") + queue_name = result.method.queue + consumer.queue_bind(exchange="custom", queue=queue_name, routing_key="custom1") + + instance.query( + f"INSERT INTO test.rabbit select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + insert_messages = [] + + def onReceived(channel, method, properties, body): + insert_messages.append(body.decode()) + if len(insert_messages) == 2: + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + assert len(insert_messages) == 2 + + assert ( + "".join(insert_messages) + == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" + ) + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view") + assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" + + +def test_row_based_formats(rabbitmq_cluster): + num_rows = 10 + + for format_name in [ + "TSV", + "TSVWithNamesAndTypes", + "TSKV", + "CSV", + "CSVWithNamesAndTypes", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "Avro", + "RowBinary", + "RowBinaryWithNamesAndTypes", + "MsgPack", + ]: + print(format_name) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbit; + + CREATE TABLE test.rabbit (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_format = '{format_name}', + rabbitmq_exchange_name = '{format_name}', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = '{format_name}', + rabbitmq_max_rows_per_message = 5; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.rabbit; + """ + ) + + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + result = consumer.queue_declare(queue="") + queue_name = result.method.queue + consumer.queue_bind( + exchange=format_name, queue=queue_name, routing_key=format_name + ) + + instance.query( + f"INSERT INTO test.rabbit SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows});" + ) + + insert_messages = 0 + + def onReceived(channel, method, properties, body): + nonlocal insert_messages + insert_messages += 1 + if insert_messages == 2: + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + assert insert_messages == 2 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + + result = instance.query("SELECT * FROM test.view") + assert result == expected + + +def test_block_based_formats_1(rabbitmq_cluster): + instance.query( + """ + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'PrettySpace', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'PrettySpace', + rabbitmq_format = 'PrettySpace'; + """ + ) + + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials + ) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + result = consumer.queue_declare(queue="") + queue_name = result.method.queue + consumer.queue_bind( + exchange="PrettySpace", queue=queue_name, routing_key="PrettySpace" + ) + + instance.query( + "INSERT INTO test.rabbitmq SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0;" + ) + insert_messages = [] + + def onReceived(channel, method, properties, body): + insert_messages.append(body.decode()) + if len(insert_messages) == 3: + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + assert len(insert_messages) == 3 + + data = [] + for message in insert_messages: + splitted = message.split("\n") + assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" + assert splitted[1] == "" + assert splitted[-1] == "" + data += [line.split() for line in splitted[2:-1]] + + assert data == [ + ["0", "0"], + ["10", "100"], + ["20", "200"], + ["30", "300"], + ["40", "400"], + ] + + +def test_block_based_formats_2(rabbitmq_cluster): + num_rows = 100 + + for format_name in [ + "JSONColumns", + "Native", + "Arrow", + "Parquet", + "ORC", + "JSONCompactColumns", + ]: + + print(format_name) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbit; + + CREATE TABLE test.rabbit (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_format = '{format_name}', + rabbitmq_exchange_name = '{format_name}', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = '{format_name}'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.rabbit; + """ + ) + + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + result = consumer.queue_declare(queue="") + queue_name = result.method.queue + consumer.queue_bind( + exchange=format_name, queue=queue_name, routing_key=format_name + ) + + instance.query( + f"INSERT INTO test.rabbit SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;" + ) + + insert_messages = 0 + + def onReceived(channel, method, properties, body): + nonlocal insert_messages + insert_messages += 1 + if insert_messages == 9: + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + assert insert_messages == 9 + + attempt = 0 + rows = 0 + while attempt < 100: + rows = int(instance.query("SELECT count() FROM test.view")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + result = instance.query("SELECT * FROM test.view ORDER by key") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected diff --git a/tests/queries/0_stateless/01420_format_row.reference b/tests/queries/0_stateless/01420_format_row.reference index 2c6dbc4d2df..560e1e038cb 100644 --- a/tests/queries/0_stateless/01420_format_row.reference +++ b/tests/queries/0_stateless/01420_format_row.reference @@ -1,56 +1,117 @@ CSV +formatRow +0,"good"\n +1,"good"\n +2,"good"\n +formatRowNoNewline 0,"good" 1,"good" 2,"good" TSV +formatRow +0\tgood\n +1\tgood\n +2\tgood\n +formatRowNoNewline 0\tgood 1\tgood 2\tgood JSONEachRow -{"number":"0","\'good\'":"good"} -{"number":"1","\'good\'":"good"} -{"number":"2","\'good\'":"good"} -JSONEachRowWithProgress -{"progress":{"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0"}}\n{"row":{"number":"0","\'good\'":"good"}} -{"row":{"number":"1","\'good\'":"good"}} -{"row":{"number":"2","\'good\'":"good"}} +formatRow +{"number":"0","good":"good"}\n +{"number":"1","good":"good"}\n +{"number":"2","good":"good"}\n +formatRowNoNewline +{"number":"0","good":"good"} +{"number":"1","good":"good"} +{"number":"2","good":"good"} JSONCompactEachRow +formatRow +["0", "good"]\n +["1", "good"]\n +["2", "good"]\n +formatRowNoNewline ["0", "good"] ["1", "good"] ["2", "good"] TSKV -number=0\t\'good\'=good -number=1\t\'good\'=good -number=2\t\'good\'=good +formatRow +number=0\tgood=good\n +number=1\tgood=good\n +number=2\tgood=good\n +formatRowNoNewline +number=0\tgood=good +number=1\tgood=good +number=2\tgood=good XML -\t\t\n\t\t\t0\n\t\t\tgood\n\t\t -\t\t\n\t\t\t1\n\t\t\tgood\n\t\t -\t\t\n\t\t\t2\n\t\t\tgood\n\t\t +formatRow +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t0\n\t\t\tgood\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t1\n\t\t\tgood\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t2\n\t\t\tgood\n\t\t\n\t\n\t1\n\n +formatRowNoNewline +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t0\n\t\t\tgood\n\t\t\n\t\n\t1\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t1\n\t\t\tgood\n\t\t\n\t\n\t1\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\tgood\n\t\t\t\tString\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t2\n\t\t\tgood\n\t\t\n\t\n\t1\n Markdown -| 0 | good | -| 1 | good | -| 2 | good | +formatRow +| number | good |\n|-:|:-|\n| 0 | good |\n +| number | good |\n|-:|:-|\n| 1 | good |\n +| number | good |\n|-:|:-|\n| 2 | good |\n +formatRowNoNewline +| number | good |\n|-:|:-|\n| 0 | good | +| number | good |\n|-:|:-|\n| 1 | good | +| number | good |\n|-:|:-|\n| 2 | good | CustomSeparated +formatRow 0\tgood\n 1\tgood\n 2\tgood\n +formatRowNoNewline +0\tgood +1\tgood +2\tgood SQLInsert -INSERT INTO table (`number`, `\'good\'`) VALUES (0, \'good\'); -INSERT INTO table (`number`, `\'good\'`) VALUES (1, \'good\'); -INSERT INTO table (`number`, `\'good\'`) VALUES (2, \'good\'); +formatRow +INSERT INTO table (`number`, `good`) VALUES (0, \'good\');\n +INSERT INTO table (`number`, `good`) VALUES (1, \'good\');\n +INSERT INTO table (`number`, `good`) VALUES (2, \'good\');\n +formatRowNoNewline +INSERT INTO table (`number`, `good`) VALUES (0, \'good\'); +INSERT INTO table (`number`, `good`) VALUES (1, \'good\'); +INSERT INTO table (`number`, `good`) VALUES (2, \'good\'); Vertical -Row 1:\n──────\nnumber: 0\n\'good\': good\n -Row 2:\n──────\nnumber: 1\n\'good\': good\n -Row 3:\n──────\nnumber: 2\n\'good\': good\n +formatRow +Row 1:\n──────\nnumber: 0\ngood: good\n +Row 1:\n──────\nnumber: 1\ngood: good\n +Row 1:\n──────\nnumber: 2\ngood: good\n +formatRowNoNewline +Row 1:\n──────\nnumber: 0\ngood: good +Row 1:\n──────\nnumber: 1\ngood: good +Row 1:\n──────\nnumber: 2\ngood: good JSON -\t\t{\n\t\t\t"number": "0",\n\t\t\t"\'good\'": "good"\n\t\t} -\t\t{\n\t\t\t"number": "1",\n\t\t\t"\'good\'": "good"\n\t\t} -\t\t{\n\t\t\t"number": "2",\n\t\t\t"\'good\'": "good"\n\t\t} +formatRow +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "0",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n}\n +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "1",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n}\n +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "2",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n}\n +formatRowNoNewline +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "0",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n} +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "1",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n} +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t{\n\t\t\t"number": "2",\n\t\t\t"good": "good"\n\t\t}\n\t],\n\n\t"rows": 1\n} JSONCompact -\t\t["0", "good"] -\t\t["1", "good"] -\t\t["2", "good"] +formatRow +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["0", "good"]\n\t],\n\n\t"rows": 1\n}\n +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["1", "good"]\n\t],\n\n\t"rows": 1\n}\n +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["2", "good"]\n\t],\n\n\t"rows": 1\n}\n +formatRowNoNewline +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["0", "good"]\n\t],\n\n\t"rows": 1\n} +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["1", "good"]\n\t],\n\n\t"rows": 1\n} +{\n\t"meta":\n\t[\n\t\t{\n\t\t\t"name": "number",\n\t\t\t"type": "UInt64"\n\t\t},\n\t\t{\n\t\t\t"name": "good",\n\t\t\t"type": "String"\n\t\t}\n\t],\n\n\t"data":\n\t[\n\t\t["2", "good"]\n\t],\n\n\t"rows": 1\n} Values +formatRow (0,\'good\') (1,\'good\') (2,\'good\') +formatRowNoNewline +(0,\'good\' +(1,\'good\' +(2,\'good\' diff --git a/tests/queries/0_stateless/01420_format_row.sql.j2 b/tests/queries/0_stateless/01420_format_row.sql.j2 index 8d6385d3a37..98a6858f2a9 100644 --- a/tests/queries/0_stateless/01420_format_row.sql.j2 +++ b/tests/queries/0_stateless/01420_format_row.sql.j2 @@ -1,14 +1,17 @@ -- Tags: no-fasttest -{% for format in ['CSV', 'TSV', 'JSONEachRow', 'JSONEachRowWithProgress', 'JSONCompactEachRow', 'TSKV', 'XML', 'Markdown', 'CustomSeparated', 'SQLInsert', 'Vertical', 'JSON', 'JSONCompact', 'Values'] -%} +set output_format_write_statistics=0; + +{% for format in ['CSV', 'TSV', 'JSONEachRow', 'JSONCompactEachRow', 'TSKV', 'XML', 'Markdown', 'CustomSeparated', 'SQLInsert', 'Vertical', 'JSON', 'JSONCompact', 'Values'] -%} select '{{ format }}'; -select formatRow('{{ format }}', number, 'good') from numbers(3); +select 'formatRow'; +select formatRow('{{ format }}', number, good) from (select number, 'good' as good from numbers(3)); +select 'formatRowNoNewline'; +select formatRowNoNewline('{{ format }}', number, good) from (select number, 'good' as good from numbers(3)); {% endfor -%} -- unknown format select formatRow('aaa', *) from numbers(3); -- { serverError 73 } -select formatRow('Avro', *) from numbers(3); -- {serverError BAD_ARGUMENTS} - diff --git a/tests/queries/0_stateless/02113_format_row.reference b/tests/queries/0_stateless/02113_format_row.reference index 33016c54af1..d137fad60f5 100644 --- a/tests/queries/0_stateless/02113_format_row.reference +++ b/tests/queries/0_stateless/02113_format_row.reference @@ -1,20 +1,20 @@ -0\t1970-01-01 -1\t1970-01-02 -2\t1970-01-03 -3\t1970-01-04 -4\t1970-01-05 -0,"1970-01-01" -1,"1970-01-02" -2,"1970-01-03" -3,"1970-01-04" -4,"1970-01-05" -["0", "1970-01-01"] -["1", "1970-01-02"] -["2", "1970-01-03"] -["3", "1970-01-04"] -["4", "1970-01-05"] -\t\t\n\t\t\t0\n\t\t\t1970-01-01\n\t\t -\t\t\n\t\t\t1\n\t\t\t1970-01-02\n\t\t -\t\t\n\t\t\t2\n\t\t\t1970-01-03\n\t\t -\t\t\n\t\t\t3\n\t\t\t1970-01-04\n\t\t -\t\t\n\t\t\t4\n\t\t\t1970-01-05\n\t\t +number\ttoDate(number)\nUInt64\tDate\n0\t1970-01-01\n +number\ttoDate(number)\nUInt64\tDate\n1\t1970-01-02\n +number\ttoDate(number)\nUInt64\tDate\n2\t1970-01-03\n +number\ttoDate(number)\nUInt64\tDate\n3\t1970-01-04\n +number\ttoDate(number)\nUInt64\tDate\n4\t1970-01-05\n +"number","toDate(number)"\n"UInt64","Date"\n0,"1970-01-01"\n +"number","toDate(number)"\n"UInt64","Date"\n1,"1970-01-02"\n +"number","toDate(number)"\n"UInt64","Date"\n2,"1970-01-03"\n +"number","toDate(number)"\n"UInt64","Date"\n3,"1970-01-04"\n +"number","toDate(number)"\n"UInt64","Date"\n4,"1970-01-05"\n +["number", "toDate(number)"]\n["UInt64", "Date"]\n["0", "1970-01-01"]\n +["number", "toDate(number)"]\n["UInt64", "Date"]\n["1", "1970-01-02"]\n +["number", "toDate(number)"]\n["UInt64", "Date"]\n["2", "1970-01-03"]\n +["number", "toDate(number)"]\n["UInt64", "Date"]\n["3", "1970-01-04"]\n +["number", "toDate(number)"]\n["UInt64", "Date"]\n["4", "1970-01-05"]\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\ttoDate(number)\n\t\t\t\tDate\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t0\n\t\t\t1970-01-01\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\ttoDate(number)\n\t\t\t\tDate\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t1\n\t\t\t1970-01-02\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\ttoDate(number)\n\t\t\t\tDate\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t2\n\t\t\t1970-01-03\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\ttoDate(number)\n\t\t\t\tDate\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t3\n\t\t\t1970-01-04\n\t\t\n\t\n\t1\n\n +\n\n\t\n\t\t\n\t\t\t\n\t\t\t\tnumber\n\t\t\t\tUInt64\n\t\t\t\n\t\t\t\n\t\t\t\ttoDate(number)\n\t\t\t\tDate\n\t\t\t\n\t\t\n\t\n\t\n\t\t\n\t\t\t4\n\t\t\t1970-01-05\n\t\t\n\t\n\t1\n\n diff --git a/tests/queries/0_stateless/02113_format_row.sql b/tests/queries/0_stateless/02113_format_row.sql index 93ee6d0f1dd..1af6f7cc740 100644 --- a/tests/queries/0_stateless/02113_format_row.sql +++ b/tests/queries/0_stateless/02113_format_row.sql @@ -1,3 +1,4 @@ +set output_format_write_statistics=0; select formatRow('TSVWithNamesAndTypes', number, toDate(number)) from numbers(5); select formatRow('CSVWithNamesAndTypes', number, toDate(number)) from numbers(5); select formatRow('JSONCompactEachRowWithNamesAndTypes', number, toDate(number)) from numbers(5); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 92c300e4694..c7ac00ee18f 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -340,6 +340,7 @@ formatReadableQuantity formatReadableSize formatReadableTimeDelta formatRow +formatRowNoNewline fragment fromModifiedJulianDay fromModifiedJulianDayOrNull diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 28c608171c9..5c8589dfca1 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -20,7 +20,7 @@ class client(object): self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) - self.client.timeout(5) + self.client.timeout(120) self.client.expect("[#\$] ", timeout=60) self.client.send(command)