From b602d18dd8c98c372dfb087ecfbdaf090e3ff07c Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 23 Sep 2020 17:10:04 +0200 Subject: [PATCH 01/10] ProtobufSingle input format Allows to read Single protobuf message at once (w/o length-delemeters). --- src/Core/Settings.h | 14 +++++-- src/Formats/ProtobufReader.cpp | 39 ++++++++++++++----- src/Formats/ProtobufReader.h | 6 ++- .../Formats/Impl/ProtobufRowInputFormat.cpp | 26 +++++++------ .../Formats/Impl/ProtobufRowInputFormat.h | 11 ++++-- tests/integration/test_storage_kafka/test.py | 35 +++++++++++++++++ 6 files changed, 100 insertions(+), 31 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a1a7a690e40..a7a7bd6d4b7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -398,7 +398,12 @@ class IColumn; M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ - M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) + M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ + M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ + M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) + +// End of COMMON_SETTINGS +// !!! please add settings related to formats into the FORMAT_FACTORY_SETTINGS below !!! #define FORMAT_FACTORY_SETTINGS(M) \ M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ @@ -462,9 +467,10 @@ class IColumn; \ M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ - M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ - M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ - M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ + M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) + +// End of FORMAT_FACTORY_SETTINGS +// !!! please add settings non-related to formats into the COMMON_SETTINGS above !!! #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 70942ddd031..e4bdd45caf8 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -38,36 +38,47 @@ namespace // Those inequations helps checking conditions in ProtobufReader::SimpleReader. constexpr Int64 END_OF_VARINT = -1; constexpr Int64 END_OF_GROUP = -2; + constexpr Int64 END_OF_FILE = -3; Int64 decodeZigZag(UInt64 n) { return static_cast((n >> 1) ^ (~(n & 1) + 1)); } - [[noreturn]] void throwUnknownFormat() - { - throw Exception("Protobuf messages are corrupted or don't match the provided schema. Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint.", ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); - } } // SimpleReader is an utility class to deserialize protobufs. // Knows nothing about protobuf schemas, just provides useful functions to deserialize data. -ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_) +ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_, const bool single_message_mode_) : in(in_) , cursor(0) , current_message_level(0) , current_message_end(0) , field_end(0) , last_string_pos(-1) + , single_message_mode(single_message_mode_) { } +[[noreturn]] void ProtobufReader::SimpleReader::throwUnknownFormat() +{ + throw Exception(std::string("Protobuf messages are corrupted or don't match the provided schema.") + (single_message_mode ? "" : " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint."), ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); +} + bool ProtobufReader::SimpleReader::startMessage() { // Start reading a root message. assert(!current_message_level); if (unlikely(in.eof())) return false; - size_t size_of_message = readVarint(); - current_message_end = cursor + size_of_message; + + if (single_message_mode) + { + current_message_end = END_OF_FILE; + } + else + { + size_t size_of_message = readVarint(); + current_message_end = cursor + size_of_message; + } ++current_message_level; field_end = cursor; return true; @@ -150,7 +161,15 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number) throwUnknownFormat(); } - if ((cursor >= current_message_end) && (current_message_end != END_OF_GROUP)) + if (current_message_end == END_OF_FILE) + { + if (unlikely(in.eof())) + { + current_message_end = cursor; + return false; + } + } + else if ((cursor >= current_message_end) && (current_message_end != END_OF_GROUP)) return false; UInt64 varint = readVarint(); @@ -1077,8 +1096,8 @@ std::unique_ptr ProtobufReader::createConverter & column_names) - : simple_reader(in_) + ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_) + : simple_reader(in_, single_message_mode_) { root_message = ProtobufColumnMatcher::matchColumns(column_names, message_type); setTraitsDataAfterMatchingColumns(root_message.get()); diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index 0e43965a6ce..39e0010e87f 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -37,7 +37,7 @@ using AggregateFunctionPtr = std::shared_ptr; class ProtobufReader : private boost::noncopyable { public: - ProtobufReader(ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names); + ProtobufReader(ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_); ~ProtobufReader(); /// Should be called when we start reading a new message. @@ -93,7 +93,7 @@ private: class SimpleReader { public: - SimpleReader(ReadBuffer & in_); + SimpleReader(ReadBuffer & in_, const bool single_message_mode_); bool startMessage(); void endMessage(bool ignore_errors); void startNestedMessage(); @@ -126,6 +126,7 @@ private: UInt64 continueReadingVarint(UInt64 first_byte); void ignoreVarint(); void ignoreGroup(); + [[noreturn]] void throwUnknownFormat(); ReadBuffer & in; Int64 cursor; @@ -134,6 +135,7 @@ private: std::vector parent_message_ends; Int64 field_end; Int64 last_string_pos; + const bool single_message_mode; }; class IConverter diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 50d9cef9bfc..ff37efc378e 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -11,10 +11,10 @@ namespace DB { -ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_) +ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool single_message_mode_) : IRowInputFormat(header_, in_, params_) , data_types(header_.getDataTypes()) - , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames()) + , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames(), single_message_mode_) { } @@ -67,16 +67,20 @@ void ProtobufRowInputFormat::syncAfterError() void registerInputFormatProcessorProtobuf(FormatFactory & factory) { - factory.registerInputFormatProcessor("Protobuf", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) + for (bool single_message_mode : {false, true}) { - return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, - settings.schema.is_server, settings.schema.format_schema_path)); - }); + factory.registerInputFormatProcessor(single_message_mode ? "ProtobufSingle" : "Protobuf", [single_message_mode]( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, + settings.schema.is_server, settings.schema.format_schema_path), + single_message_mode); + }); + } } } diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 8dd36722efc..7f6a7ead0c2 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -16,17 +16,20 @@ class FormatSchemaInfo; /** Stream designed to deserialize data from the google protobuf format. - * Each row is read as a separated message. - * These messages are delimited according to documentation + * One Protobuf message is parsed as one row of data. + * + * Input buffer may contain single protobuf message (single_message_mode_ = true), + * or any number of messages (single_message_mode_ = false). In the second case + * parser assumes messages are length-delimited according to documentation * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h - * Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g. + * Parsing of the protobuf format requires the 'format_schema' setting to be set, e.g. * INSERT INTO table FORMAT Protobuf SETTINGS format_schema = 'schema:Message' * where schema is the name of "schema.proto" file specifying protobuf schema. */ class ProtobufRowInputFormat : public IRowInputFormat { public: - ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_); + ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool single_message_mode_); ~ProtobufRowInputFormat() override; String getName() const override { return "ProtobufRowInputFormat"; } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 69ba09899ed..32e00276bb6 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -103,6 +103,17 @@ def kafka_produce_protobuf_messages(topic, start_index, num_messages): producer.flush() print(("Produced {} messages for topic {}".format(num_messages, topic))) +def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messages): + data = '' + producer = KafkaProducer(bootstrap_servers="localhost:9092") + for i in range(start_index, start_index + num_messages): + msg = kafka_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + producer.send(topic=topic, value=serialized_msg) + producer.flush() + print("Produced {} messages for topic {}".format(num_messages, topic)) def avro_confluent_message(schema_registry_client, value): # type: (CachedSchemaRegistryClient, dict) -> str @@ -971,6 +982,30 @@ def test_kafka_protobuf(kafka_cluster): kafka_check_result(result, True) +@pytest.mark.timeout(180) +def test_kafka_protobuf_no_delimiter(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'pb_no_delimiter', + kafka_group_name = 'pb_no_delimiter', + kafka_format = 'ProtobufSingle', + kafka_schema = 'kafka.proto:KeyValuePair'; + ''') + + kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 0, 20) + kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 20, 1) + kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 21, 29) + + result = '' + while True: + result += instance.query('SELECT * FROM test.kafka', ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + @pytest.mark.timeout(180) def test_kafka_materialized_view(kafka_cluster): instance.query(''' From 6df42c580bf4f4e75e3bdbfdeab815017b5394ac Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 23 Sep 2020 19:16:04 +0200 Subject: [PATCH 02/10] ProtobufSingle output format --- src/Formats/ProtobufReader.cpp | 2 +- src/Formats/ProtobufReader.h | 2 +- src/Formats/ProtobufWriter.cpp | 28 +++++++++++++--- src/Formats/ProtobufWriter.h | 8 +++-- src/IO/WriteBuffer.h | 5 +++ .../Formats/Impl/ProtobufRowOutputFormat.cpp | 32 +++++++++++-------- .../Formats/Impl/ProtobufRowOutputFormat.h | 3 +- .../Kafka/WriteBufferToKafkaProducer.h | 2 ++ .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 1 + 9 files changed, 60 insertions(+), 23 deletions(-) diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index e4bdd45caf8..d12e64050bb 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -58,7 +58,7 @@ ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_, const bool single_m { } -[[noreturn]] void ProtobufReader::SimpleReader::throwUnknownFormat() +[[noreturn]] void ProtobufReader::SimpleReader::throwUnknownFormat() const { throw Exception(std::string("Protobuf messages are corrupted or don't match the provided schema.") + (single_message_mode ? "" : " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint."), ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); } diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index 39e0010e87f..fd9da20781e 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -126,7 +126,7 @@ private: UInt64 continueReadingVarint(UInt64 first_byte); void ignoreVarint(); void ignoreGroup(); - [[noreturn]] void throwUnknownFormat(); + [[noreturn]] void throwUnknownFormat() const; ReadBuffer & in; Int64 cursor; diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index b03500437b1..3f9df36329b 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -21,6 +21,7 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; extern const int PROTOBUF_BAD_CAST; extern const int PROTOBUF_FIELD_NOT_REPEATED; + extern const int TOO_MANY_ROWS; } @@ -123,7 +124,16 @@ namespace // SimpleWriter is an utility class to serialize protobufs. // Knows nothing about protobuf schemas, just provides useful functions to serialize data. -ProtobufWriter::SimpleWriter::SimpleWriter(WriteBuffer & out_) : out(out_), current_piece_start(0), num_bytes_skipped(0) +ProtobufWriter::SimpleWriter::SimpleWriter(WriteBuffer & out_, const bool single_message_mode_) + : out(out_) + , current_piece_start(0) + , num_bytes_skipped(0) + , produce_length_delimiters(!single_message_mode_) + // normally for single_message_mode we forbid outputting more than one row + // (it would lead to malformed protobuf message), but Kafka/Rabbit push every row + // in a separate message, so it's valid case there and should be allowed. + , allow_several_messages(!single_message_mode_ || out.producesIsolatedRows()) + , row_was_send(false) { } @@ -131,13 +141,21 @@ ProtobufWriter::SimpleWriter::~SimpleWriter() = default; void ProtobufWriter::SimpleWriter::startMessage() { + if (!allow_several_messages && row_was_send) + { + throw Exception("ProtobufSingle can output only single row at a time.", ErrorCodes::TOO_MANY_ROWS); + } + row_was_send = true; } void ProtobufWriter::SimpleWriter::endMessage() { pieces.emplace_back(current_piece_start, buffer.size()); - size_t size_of_message = buffer.size() - num_bytes_skipped; - writeVarint(size_of_message, out); + if (produce_length_delimiters) + { + size_t size_of_message = buffer.size() - num_bytes_skipped; + writeVarint(size_of_message, out); + } for (const auto & piece : pieces) if (piece.end > piece.start) out.write(reinterpret_cast(&buffer[piece.start]), piece.end - piece.start); @@ -827,8 +845,8 @@ std::unique_ptr ProtobufWriter::createConverter & column_names) - : simple_writer(out) + WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_) + : simple_writer(out, single_message_mode_) { std::vector field_descriptors_without_match; root_message = ProtobufColumnMatcher::matchColumns(column_names, message_type, field_descriptors_without_match); diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 10bbbb14976..c49fe6e8edf 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -37,7 +37,7 @@ using ConstAggregateDataPtr = const char *; class ProtobufWriter : private boost::noncopyable { public: - ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names); + ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_); ~ProtobufWriter(); /// Should be called at the beginning of writing a message. @@ -89,7 +89,7 @@ private: class SimpleWriter { public: - SimpleWriter(WriteBuffer & out_); + SimpleWriter(WriteBuffer & out_, const bool single_message_mode_); ~SimpleWriter(); void startMessage(); @@ -138,6 +138,10 @@ private: size_t current_piece_start; size_t num_bytes_skipped; std::vector nested_infos; + const bool produce_length_delimiters; + const bool allow_several_messages; + bool row_was_send; + }; class IConverter diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index b8bf53e18fd..6de7534eeff 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -60,6 +60,11 @@ public: */ virtual ~WriteBuffer() {} + /** + * some buffers (kafka / rabbit) split the rows internally, so we can push there formats without + * framing / delimiters (like ProtobufSingle) + */ + virtual bool producesIsolatedRows() { return false; } inline void nextIfAtEnd() { diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index a9efd08db51..2a820a7f519 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -21,10 +21,11 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, - const FormatSchemaInfo & format_schema) + const FormatSchemaInfo & format_schema, + const bool single_message_mode_) : IRowOutputFormat(header, out_, callback) , data_types(header.getDataTypes()) - , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames()) + , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), single_message_mode_) { value_indices.resize(header.columns()); } @@ -43,17 +44,22 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { - factory.registerOutputFormatProcessor( - "Protobuf", - [](WriteBuffer & buf, - const Block & header, - FormatFactory::WriteCallback callback, - const FormatSettings & settings) - { - return std::make_shared(buf, header, std::move(callback), - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, - settings.schema.is_server, settings.schema.format_schema_path)); - }); + for (bool single_message_mode : {false, true}) + { + + factory.registerOutputFormatProcessor( + single_message_mode ? "ProtobufSingle" : "Protobuf", + [single_message_mode](WriteBuffer & buf, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings) + { + return std::make_shared(buf, header, std::move(callback), + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, + settings.schema.is_server, settings.schema.format_schema_path), + single_message_mode); + }); + } } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index f216af01d80..b5bfb06db5a 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -38,7 +38,8 @@ public: WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, - const FormatSchemaInfo & format_schema); + const FormatSchemaInfo & format_schema, + const bool single_message_mode_); String getName() const override { return "ProtobufRowOutputFormat"; } diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/src/Storages/Kafka/WriteBufferToKafkaProducer.h index 76859c4e33f..547d796b576 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.h +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.h @@ -25,6 +25,8 @@ public: const Block & header); ~WriteBufferToKafkaProducer() override; + bool producesIsolatedRows() override { return true; } + void countRow(const Columns & columns, size_t row); void flush(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index a8e94070dbd..50ec4b518e1 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -34,6 +34,7 @@ public: ); ~WriteBufferToRabbitMQProducer() override; + bool producesIsolatedRows() override { return true; } void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } From e17d4e1eeb14fb4ba7e9ea02604b7438c0551e6e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 25 Sep 2020 18:17:57 +0200 Subject: [PATCH 03/10] Tests and some docs --- docs/en/interfaces/formats.md | 5 ++++ docs/ru/interfaces/formats.md | 9 ++++-- .../Formats/Impl/ProtobufRowOutputFormat.h | 4 ++- tests/integration/test_storage_kafka/test.py | 27 +++++++++++++++++- .../00825_protobuf_format_input.reference | 7 +++++ .../00825_protobuf_format_input.sh | 15 ++++++++++ .../00825_protobuf_format_input_single.insh | 12 ++++++++ .../00825_protobuf_format_output.reference | Bin 2433 -> 3726 bytes .../00825_protobuf_format_output.sh | 17 ++++++++++- 9 files changed, 91 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/00825_protobuf_format_input_single.insh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index bfe5b6218e4..9fe57b5d57b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -43,6 +43,7 @@ The supported formats are: | [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | | [PrettySpace](#prettyspace) | ✗ | ✔ | | [Protobuf](#protobuf) | ✔ | ✔ | +| [ProtobufSingle](#protobufsingle) | ✔ | ✔ | | [Avro](#data-format-avro) | ✔ | ✔ | | [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | | [Parquet](#data-format-parquet) | ✔ | ✔ | @@ -1075,6 +1076,10 @@ ClickHouse inputs and outputs protobuf messages in the `length-delimited` format It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints). See also [how to read/write length-delimited protobuf messages in popular languages](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages). +## ProtobufSingle {#protobufsingle} + +Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters. + ## Avro {#data-format-avro} [Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index dd68f7eb646..02fcd8a20d4 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -27,6 +27,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | | [PrettySpace](#prettyspace) | ✗ | ✔ | | [Protobuf](#protobuf) | ✔ | ✔ | +| [ProtobufSingle](#protobufsingle) | ✔ | ✔ | | [Parquet](#data-format-parquet) | ✔ | ✔ | | [Arrow](#data-format-arrow) | ✔ | ✔ | | [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | @@ -947,6 +948,10 @@ message MessageType { ClickHouse пишет и читает сообщения `Protocol Buffers` в формате `length-delimited`. Это означает, что перед каждым сообщением пишется его длина в формате [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints). См. также [как читать и записывать сообщения Protocol Buffers в формате length-delimited в различных языках программирования](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages). +## ProtobufSingle {#protobufsingle} + +То же, что [Protobuf](#protobuf), но без разделителей. Позволяет записать / прочитать не более одного сообщения за раз. + ## Avro {#data-format-avro} [Apache Avro](https://avro.apache.org/) — это ориентированный на строки фреймворк для сериализации данных. Разработан в рамках проекта Apache Hadoop. @@ -957,7 +962,7 @@ ClickHouse пишет и читает сообщения `Protocol Buffers` в ## AvroConfluent {#data-format-avro-confluent} -Для формата `AvroConfluent` ClickHouse поддерживает декодирование сообщений `Avro` с одним объектом. Такие сообщения используются с [Kafka] (http://kafka.apache.org/) и реестром схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html). +Для формата `AvroConfluent` ClickHouse поддерживает декодирование сообщений `Avro` с одним объектом. Такие сообщения используются с [Kafka] (http://kafka.apache.org/) и реестром схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html). Каждое сообщение `Avro` содержит идентификатор схемы, который может быть разрешен для фактической схемы с помощью реестра схем. @@ -971,7 +976,7 @@ URL-адрес реестра схем настраивается с помощ ### Использование {#ispolzovanie} -Чтобы быстро проверить разрешение схемы, используйте [kafkacat](https://github.com/edenhill/kafkacat) с языком запросов [clickhouse-local](../operations/utilities/clickhouse-local.md): +Чтобы быстро проверить разрешение схемы, используйте [kafkacat](https://github.com/edenhill/kafkacat) с языком запросов [clickhouse-local](../operations/utilities/clickhouse-local.md): ``` bash $ kafkacat -b kafka-broker -C -t topic1 -o beginning -f '%s' -c 3 | clickhouse-local --input-format AvroConfluent --format_avro_schema_registry_url 'http://schema-registry' -S "field1 Int64, field2 String" -q 'select * from table' diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index b5bfb06db5a..5a30f22a59f 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -25,7 +25,9 @@ namespace DB { /** Stream designed to serialize data in the google protobuf format. * Each row is written as a separated message. - * These messages are delimited according to documentation + * + * With single_message_mode=1 it can write only single row as plain protobuf message, + * otherwise Protobuf messages are delimited according to documentation * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h * Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g. * SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message' diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 32e00276bb6..6ef37c1e231 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -982,7 +982,7 @@ def test_kafka_protobuf(kafka_cluster): kafka_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(30) def test_kafka_protobuf_no_delimiter(kafka_cluster): instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) @@ -1006,6 +1006,31 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_check_result(result, True) + instance.query(''' + CREATE TABLE test.kafka_writer (key UInt64, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'pb_no_delimiter', + kafka_group_name = 'pb_no_delimiter', + kafka_format = 'ProtobufSingle', + kafka_schema = 'kafka.proto:KeyValuePair'; + ''') + + instance.query("INSERT INTO test.kafka_writer VALUES (13,'Friday'),(42,'Answer to the Ultimate Question of Life, the Universe, and Everything'), (110, 'just a number')") + + time.sleep(1) + + result = instance.query("SELECT * FROM test.kafka ORDER BY key", ignore_error=True) + + expected = '''\ +13 Friday +42 Answer to the Ultimate Question of Life, the Universe, and Everything +110 just a number +''' + assert TSV(result) == TSV(expected) + + + @pytest.mark.timeout(180) def test_kafka_materialized_view(kafka_cluster): instance.query(''' diff --git a/tests/queries/0_stateless/00825_protobuf_format_input.reference b/tests/queries/0_stateless/00825_protobuf_format_input.reference index 0c56bc4ebf0..75fef3f8ac3 100644 --- a/tests/queries/0_stateless/00825_protobuf_format_input.reference +++ b/tests/queries/0_stateless/00825_protobuf_format_input.reference @@ -8,4 +8,11 @@ a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +7495123456 0 0 2 4 3 9 +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 [] +2 4 +3 9 +ok ok diff --git a/tests/queries/0_stateless/00825_protobuf_format_input.sh b/tests/queries/0_stateless/00825_protobuf_format_input.sh index 66c92e6fb35..b9912b2b849 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_input.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_input.sh @@ -48,6 +48,14 @@ source "$CURDIR"/00825_protobuf_format_input.insh $CLICKHOUSE_CLIENT --query "SELECT * FROM in_persons_00825 ORDER BY uuid;" $CLICKHOUSE_CLIENT --query "SELECT * FROM in_squares_00825 ORDER BY number;" +$CLICKHOUSE_CLIENT --query "TRUNCATE TABLE in_persons_00825;" +$CLICKHOUSE_CLIENT --query "TRUNCATE TABLE in_squares_00825;" + +source "$CURDIR"/00825_protobuf_format_input_single.insh + +$CLICKHOUSE_CLIENT --query "SELECT * FROM in_persons_00825 ORDER BY uuid;" +$CLICKHOUSE_CLIENT --query "SELECT * FROM in_squares_00825 ORDER BY number;" + # Try to input malformed data. set +eo pipefail echo -ne '\xe0\x80\x3f\x0b' \ @@ -55,5 +63,12 @@ echo -ne '\xe0\x80\x3f\x0b' \ | grep -qF "Protobuf messages are corrupted" && echo "ok" || echo "fail" set -eo pipefail +# Try to input malformed data for ProtobufSingle +set +eo pipefail +echo -ne '\xff\xff\x3f\x0b' \ + | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" 2>&1 \ + | grep -qF "Protobuf messages are corrupted" && echo "ok" || echo "fail" +set -eo pipefail + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_persons_00825;" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_squares_00825;" diff --git a/tests/queries/0_stateless/00825_protobuf_format_input_single.insh b/tests/queries/0_stateless/00825_protobuf_format_input_single.insh new file mode 100644 index 00000000000..6c4dfec05aa --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_input_single.insh @@ -0,0 +1,12 @@ +echo -ne '\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" +echo -ne '\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" +echo -ne '\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" +echo -ne '\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'" + +echo -ne '\x08\x02\x10\x04' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" +echo -ne '\x08\x03\x10\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" + +### Actually empty Protobuf message is a valid message (with all values default). +### It will work in Kafka but clickhouse-client forbids that: +### Code: 108. DB::Exception: No data to insert +## echo -ne '' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" \ No newline at end of file diff --git a/tests/queries/0_stateless/00825_protobuf_format_output.reference b/tests/queries/0_stateless/00825_protobuf_format_output.reference index 9d20d778ff6ac82cc23b20eed6ed833bb628fcb8..f0e0ac58a5e323517118e4e2137cd5bcdb5a1feb 100644 GIT binary patch delta 115 zcmZn^?vvfn$SE6>7n7%@r4Ue*Uy`3xniiaym!6ZVprr*8=bCKDo-;A8gU`_?#5Ksz zF~l>>Ro9MdvOCieuHcX$Pe1oy5Z931WO5nHXRhE#zYxa=BZ$yPuHe8>#~@dz3Xm}X DQwt;E delta 7 OcmeB^Z4}'; SELECT * FROM out_squares_00825 ORDER BY number FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; +SELECT '\n\n** ProtobufSingle **\n\n'; + +SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; +SELECT 'ALTERNATIVE->'; +SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'; +SELECT 'STRINGS->'; +SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'; +SELECT 'SYNTAX2->'; +SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'; +SELECT 'SQUARES->'; +SELECT * FROM out_squares_00825 ORDER BY number LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; + +-- Code: 158, e.displayText() = DB::Exception: ProtobufSingle can output only single row at a time. +SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 158 } + DROP TABLE IF EXISTS out_persons_00825; DROP TABLE IF EXISTS out_squares_00825; EOF From c37a456c49ffa92a19077e72b388238a4f068054 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 6 Oct 2020 14:47:52 +0200 Subject: [PATCH 04/10] Refactoring WriteCallback->RowOutputFormatParams --- src/Formats/FormatFactory.cpp | 6 +++++- src/Formats/FormatFactory.h | 3 ++- src/Processors/Formats/IRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/IRowOutputFormat.h | 18 +++++++++++++---- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 4 ++-- .../Formats/Impl/AvroRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/AvroRowOutputFormat.h | 2 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 12 +++++------ .../Formats/Impl/BinaryRowOutputFormat.h | 2 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/CSVRowOutputFormat.h | 2 +- .../JSONCompactEachRowRowOutputFormat.cpp | 20 +++++++++---------- .../Impl/JSONCompactEachRowRowOutputFormat.h | 2 +- .../Impl/JSONCompactRowOutputFormat.cpp | 12 +++++------ .../Formats/Impl/JSONCompactRowOutputFormat.h | 2 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 12 +++++------ .../Formats/Impl/JSONEachRowRowOutputFormat.h | 2 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/JSONRowOutputFormat.cpp | 12 +++++------ .../Formats/Impl/JSONRowOutputFormat.h | 2 +- .../Formats/Impl/MarkdownRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/MarkdownRowOutputFormat.h | 2 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/MsgPackRowOutputFormat.h | 2 +- .../Formats/Impl/MySQLOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 2 +- src/Processors/Formats/Impl/NullFormat.cpp | 2 +- .../Impl/ODBCDriver2BlockOutputFormat.cpp | 2 +- .../Formats/Impl/ORCBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../Formats/Impl/PostgreSQLOutputFormat.cpp | 2 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 4 ++-- .../Impl/PrettyCompactBlockOutputFormat.cpp | 4 ++-- .../Impl/PrettySpaceBlockOutputFormat.cpp | 4 ++-- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 9 ++++----- .../Formats/Impl/ProtobufRowOutputFormat.h | 2 +- .../Formats/Impl/TSKVRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/TSKVRowOutputFormat.h | 2 +- .../Impl/TabSeparatedRawRowOutputFormat.h | 4 ++-- .../Impl/TabSeparatedRowOutputFormat.cpp | 20 +++++++++---------- .../Impl/TabSeparatedRowOutputFormat.h | 2 +- .../Impl/TemplateBlockOutputFormat.cpp | 4 ++-- .../Formats/Impl/ValuesRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/ValuesRowOutputFormat.h | 2 +- .../Formats/Impl/VerticalRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/VerticalRowOutputFormat.h | 2 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 8 ++++---- .../Formats/Impl/XMLRowOutputFormat.h | 2 +- 48 files changed, 141 insertions(+), 127 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 065b14f86b7..80d45a87eda 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -268,10 +269,13 @@ OutputFormatPtr FormatFactory::getOutputFormat( const Settings & settings = context.getSettingsRef(); FormatSettings format_settings = getOutputFormatSetting(settings, context); + RowOutputFormatParams params; + params.callback = std::move(callback); + /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - auto format = output_getter(buf, sample, std::move(callback), format_settings); + auto format = output_getter(buf, sample, params, format_settings); /// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query. if (format_settings.enable_streaming) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index de53490dd3b..09fc3cdf7a5 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -27,6 +27,7 @@ class IInputFormat; class IOutputFormat; struct RowInputFormatParams; +struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; @@ -80,7 +81,7 @@ private: using OutputProcessorCreator = std::function; struct Creators diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index bb74f1ce59e..82d83a88cf1 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -25,8 +25,8 @@ void IRowOutputFormat::consume(DB::Chunk chunk) write(columns, row); - if (write_single_row_callback) - write_single_row_callback(columns, row); + if (params.callback) + params.callback(columns, row); } } diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index a702714a77d..73a2cb271fa 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -9,6 +9,15 @@ namespace DB { +/// Common parameters for generating blocks. +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,8 +33,10 @@ protected: void finalize() override; public: - IRowOutputFormat(const Block & header, WriteBuffer & out_, FormatFactory::WriteCallback callback) - : IOutputFormat(header, out_), types(header.getDataTypes()), write_single_row_callback(callback) + using Params = RowOutputFormatParams; + + IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_) + : IOutputFormat(header, out_), types(header.getDataTypes()), params(params_) { } @@ -59,8 +70,7 @@ private: bool prefix_written = false; bool suffix_written = false; - // Callback used to indicate that another row is written. - FormatFactory::WriteCallback write_single_row_callback; + Params params; void writePrefixIfNot() { diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 81f3d92b888..756172a5a68 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -79,7 +79,7 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory) "Arrow", [](WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, false, format_settings); @@ -89,7 +89,7 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory) "ArrowStream", [](WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, true, format_settings); diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 82688fe407c..74b9541d59a 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -347,8 +347,8 @@ static avro::Codec getCodec(const std::string & codec_name) } AvroRowOutputFormat::AvroRowOutputFormat( - WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_, callback) + WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & settings_) + : IRowOutputFormat(header_, out_, params) , settings(settings_) , serializer(header_.getColumnsWithTypeAndName()) , file_writer( @@ -383,10 +383,10 @@ void registerOutputFormatProcessorAvro(FormatFactory & factory) factory.registerOutputFormatProcessor("Avro", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index 4d404337d74..2f1953149e1 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -43,7 +43,7 @@ private: class AvroRowOutputFormat : public IRowOutputFormat { public: - AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & settings_); virtual ~AvroRowOutputFormat() override; String getName() const override { return "AvroRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index f228044f13d..ad734c5cfa9 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -9,8 +9,8 @@ namespace DB { -BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, FormatFactory::WriteCallback callback) - : IRowOutputFormat(header, out_, callback), with_names(with_names_), with_types(with_types_) +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_) { } @@ -52,19 +52,19 @@ void registerOutputFormatProcessorRowBinary(FormatFactory & factory) factory.registerOutputFormatProcessor("RowBinary", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, false, false, callback); + return std::make_shared(buf, sample, false, false, params); }); factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, true, true, callback); + return std::make_shared(buf, sample, true, true, params); }); } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index 0452a1e3e0f..e8467663038 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class BinaryRowOutputFormat: public IRowOutputFormat { public: - BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, FormatFactory::WriteCallback callback); + BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params); String getName() const override { return "BinaryRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 2ff8defd3b0..695ae148749 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), with_names(with_names_), format_settings(format_settings_) +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params), with_names(with_names_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -77,10 +77,10 @@ void registerOutputFormatProcessorCSV(FormatFactory & factory) factory.registerOutputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, with_names, callback, format_settings); + return std::make_shared(buf, sample, with_names, params, format_settings); }); } } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 05b097f7514..3da76f4fee5 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_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); String getName() const override { return "CSVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index ab8fd164c3c..a855c7d2215 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -10,11 +10,11 @@ namespace DB JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool with_names_, bool yield_strings_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) + : IRowOutputFormat(header_, out_, params), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -103,37 +103,37 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONCompactEachRow", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false, false); + return std::make_shared(buf, sample, params, format_settings, false, false); }); factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( WriteBuffer &buf, const Block &sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true, false); + return std::make_shared(buf, sample, params, format_settings, true, false); }); factory.registerOutputFormatProcessor("JSONCompactStringsEachRow", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false, true); + return std::make_shared(buf, sample, params, format_settings, false, true); }); factory.registerOutputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( WriteBuffer &buf, const Block &sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true, true); + return std::make_shared(buf, sample, params, format_settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index 56936783e78..5449168a1ec 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -18,7 +18,7 @@ public: JSONCompactEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool with_names_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index c36942cff09..b9d4a43d36a 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -10,10 +10,10 @@ namespace DB JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_) - : JSONRowOutputFormat(out_, header, callback, settings_, yield_strings_) + : JSONRowOutputFormat(out_, header, params, settings_, yield_strings_) { } @@ -93,19 +93,19 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONCompact", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, params, format_settings, false); }); factory.registerOutputFormatProcessor("JSONCompactStrings", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, params, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index 6585016c44f..a800537877b 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -19,7 +19,7 @@ public: JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 069499d99c1..5cc14851b28 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -11,10 +11,10 @@ namespace DB JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), yield_strings(yield_strings_) + : IRowOutputFormat(header_, out_, params), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -71,19 +71,19 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONEachRow", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, params, format_settings, false); }); factory.registerOutputFormatProcessor("JSONStringsEachRow", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, params, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 5346a1ab19f..60cdbcd921e 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -18,7 +18,7 @@ public: JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 35720df9672..48c23abd680 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -33,19 +33,19 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, params, format_settings, false); }); factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, params, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 7dd7eb9953a..c73abeaa98a 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -10,10 +10,10 @@ namespace DB JSONRowOutputFormat::JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_) - : IRowOutputFormat(header, out_, callback), settings(settings_), yield_strings(yield_strings_) + : IRowOutputFormat(header, out_, params), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -271,19 +271,19 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) factory.registerOutputFormatProcessor("JSON", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, params, format_settings, false); }); factory.registerOutputFormatProcessor("JSONStrings", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, params, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 4e9cceb717e..2861fccdac3 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -19,7 +19,7 @@ public: JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index ae793232cf4..1ca40522048 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_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) {} +MarkdownRowOutputFormat::MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params), format_settings(format_settings_) {} void MarkdownRowOutputFormat::writePrefix() { @@ -60,10 +60,10 @@ void registerOutputFormatProcessorMarkdown(FormatFactory & factory) factory.registerOutputFormatProcessor("Markdown", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h index 2f39cce0181..0b79e8c42ff 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -12,7 +12,7 @@ class ReadBuffer; class MarkdownRowOutputFormat : public IRowOutputFormat { public: - MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); /// Write higher part of markdown table like this: /// |columnName1|columnName2|...|columnNameN| diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index cc0a5f297ea..d42c4f7b002 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -24,8 +24,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback) - : IRowOutputFormat(header_, out_, callback), packer(out_) {} +MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params) + : IRowOutputFormat(header_, out_, params), packer(out_) {} void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num) { @@ -154,10 +154,10 @@ void registerOutputFormatProcessorMsgPack(FormatFactory & factory) factory.registerOutputFormatProcessor("MsgPack", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, callback); + return std::make_shared(buf, sample, params); }); } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h index 00bdfcc21cf..0f91e174367 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h @@ -20,7 +20,7 @@ namespace DB class MsgPackRowOutputFormat : public IRowOutputFormat { public: - MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback); + MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params); String getName() const override { return "MsgPackRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index b3d4198a396..067c5cb0ab9 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -95,7 +95,7 @@ void registerOutputFormatProcessorMySQLWire(FormatFactory & factory) "MySQLWire", [](WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + 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 bbc0d05272b..86fafd4e2c3 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -168,7 +168,7 @@ void registerOutputFormatProcessorNative(FormatFactory & factory) factory.registerOutputFormatProcessor("Native", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings &) { return std::make_shared(sample, buf); diff --git a/src/Processors/Formats/Impl/NullFormat.cpp b/src/Processors/Formats/Impl/NullFormat.cpp index c8c773e7acf..fda79d6981a 100644 --- a/src/Processors/Formats/Impl/NullFormat.cpp +++ b/src/Processors/Formats/Impl/NullFormat.cpp @@ -21,7 +21,7 @@ void registerOutputFormatProcessorNull(FormatFactory & factory) factory.registerOutputFormatProcessor("Null", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings &) { return std::make_shared(sample, buf); diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index b7018a1c1e8..3dd72a7a5c7 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -107,7 +107,7 @@ void ODBCDriver2BlockOutputFormat::writePrefix() void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, FormatFactory::WriteCallback, const FormatSettings & format_settings) + "ODBCDriver2", [](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.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 3bf2a9dbf59..019ffc73d81 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -433,7 +433,7 @@ void registerOutputFormatProcessorORC(FormatFactory & factory) factory.registerOutputFormatProcessor("ORC", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 695b8efa3fb..78b6a9c53a4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -84,7 +84,7 @@ void registerOutputFormatProcessorParquet(FormatFactory & factory) "Parquet", [](WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { auto impl = std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp index d5e5d97e029..50b3def929e 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp @@ -73,7 +73,7 @@ void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory) "PostgreSQLWire", [](WriteBuffer & buf, const Block & sample, - const FormatFactory::WriteCallback &, + const RowOutputFormatParams &, const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); } } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index b07cf83eb85..628a90beefb 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -403,7 +403,7 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) factory.registerOutputFormatProcessor("Pretty", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); @@ -412,7 +412,7 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettyNoEscapes", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index e832dd83388..b81ba84c732 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -259,7 +259,7 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) factory.registerOutputFormatProcessor(name, [mono_block = mono_block]( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings, mono_block); @@ -269,7 +269,7 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettyCompactNoEscapes", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index f1a5cbac8e0..f3fb27a5558 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -113,7 +113,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettySpace", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); @@ -122,7 +122,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettySpaceNoEscapes", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & format_settings) { FormatSettings changed_settings = format_settings; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 2a820a7f519..e8b7ac0fc93 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -20,10 +20,10 @@ namespace ErrorCodes ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSchemaInfo & format_schema, const bool single_message_mode_) - : IRowOutputFormat(header, out_, callback) + : IRowOutputFormat(header, out_, params) , data_types(header.getDataTypes()) , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), single_message_mode_) { @@ -46,15 +46,14 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { for (bool single_message_mode : {false, true}) { - factory.registerOutputFormatProcessor( single_message_mode ? "ProtobufSingle" : "Protobuf", [single_message_mode](WriteBuffer & buf, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, header, std::move(callback), + return std::make_shared(buf, header, params, FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, settings.schema.is_server, settings.schema.format_schema_path), single_message_mode); diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 5a30f22a59f..1183813d26d 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -39,7 +39,7 @@ public: ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSchemaInfo & format_schema, const bool single_message_mode_); diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index 17677acfb86..129cb27ce53 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, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header, false, false, callback, format_settings_) +TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out_, header, false, false, params, format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -44,10 +44,10 @@ void registerOutputFormatProcessorTSKV(FormatFactory & factory) factory.registerOutputFormatProcessor("TSKV", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 7166176e2f4..98a5846218c 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -14,7 +14,7 @@ namespace DB class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat { public: - TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & format_settings); + TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params, const FormatSettings & format_settings); String getName() const override { return "TSKVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h index 42ebd846dfc..b8b0dbc0aa6 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h @@ -18,9 +18,9 @@ public: const Block & header_, bool with_names_, bool with_types_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, callback, format_settings_) + : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, params, format_settings_) { } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 95377c13b55..f5325b3e869 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -11,9 +11,9 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( const Block & header_, bool with_names_, bool with_types_, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) + : IRowOutputFormat(header_, out_, params), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { } @@ -80,10 +80,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, false, false, callback, settings); + return std::make_shared(buf, sample, false, false, params, settings); }); } @@ -92,10 +92,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, false, false, callback, settings); + return std::make_shared(buf, sample, false, false, params, settings); }); } @@ -104,10 +104,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, false, callback, settings); + return std::make_shared(buf, sample, true, false, params, settings); }); } @@ -116,10 +116,10 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, true, callback, settings); + return std::make_shared(buf, sample, true, true, params, settings); }); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 137ffde4b18..3caf1c7003e 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -23,7 +23,7 @@ public: const Block & header_, bool with_names_, bool with_types_, - FormatFactory::WriteCallback callback, + 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 9786943a79e..6e33c7d90c9 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -232,7 +232,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) factory.registerOutputFormatProcessor("Template", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & settings) { ParsedTemplateFormatString resultset_format; @@ -270,7 +270,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) factory.registerOutputFormatProcessor("CustomSeparated", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings & settings) { ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(settings.custom); diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index 6294a829456..71dbe4da00c 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_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) +ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params), format_settings(format_settings_) { } @@ -46,10 +46,10 @@ void registerOutputFormatProcessorValues(FormatFactory & factory) factory.registerOutputFormatProcessor("Values", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h index b507a157e57..183fd44cb60 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -15,7 +15,7 @@ class WriteBuffer; class ValuesRowOutputFormat : public IRowOutputFormat { public: - ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, 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 7c1f0e3b363..c6dad489059 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_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) + WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -168,10 +168,10 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory) factory.registerOutputFormatProcessor("Vertical", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index aa40a6a5ad6..36483002325 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -18,7 +18,7 @@ class Context; class VerticalRowOutputFormat : public IRowOutputFormat { public: - VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); String getName() const override { return "VerticalRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index dba07660503..e36af1610de 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_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) +XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -245,10 +245,10 @@ void registerOutputFormatProcessorXML(FormatFactory & factory) factory.registerOutputFormatProcessor("XML", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, callback, settings); + return std::make_shared(buf, sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index 8c3da59cfd1..91be4db82f9 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -16,7 +16,7 @@ namespace DB class XMLRowOutputFormat : public IRowOutputFormat { public: - XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); String getName() const override { return "XMLRowOutputFormat"; } From f6b00f2cb6c30b081a6779782de0344d122b1d1e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 6 Oct 2020 16:32:01 +0200 Subject: [PATCH 05/10] After CR fixes --- src/Formats/FormatFactory.cpp | 7 ++-- src/Formats/FormatFactory.h | 4 +- src/Formats/ProtobufReader.cpp | 39 +++++++++++-------- src/Formats/ProtobufReader.h | 6 +-- src/Formats/ProtobufWriter.cpp | 21 +++------- src/Formats/ProtobufWriter.h | 9 ++--- src/Formats/tests/tab_separated_streams.cpp | 7 ++-- src/IO/WriteBuffer.h | 6 --- src/Processors/Formats/IRowOutputFormat.h | 11 +++++- .../Formats/Impl/ProtobufRowInputFormat.cpp | 10 ++--- .../Formats/Impl/ProtobufRowInputFormat.h | 6 +-- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 19 ++++++--- .../Formats/Impl/ProtobufRowOutputFormat.h | 5 ++- .../Formats/Impl/RawBLOBRowOutputFormat.cpp | 8 ++-- .../Formats/Impl/RawBLOBRowOutputFormat.h | 2 +- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- .../Kafka/WriteBufferToKafkaProducer.h | 2 - .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 4 +- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 1 - 19 files changed, 86 insertions(+), 83 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 80d45a87eda..22427fbfaee 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -203,7 +203,7 @@ BlockInputStreamPtr FormatFactory::getInput( BlockOutputStreamPtr FormatFactory::getOutput( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const { if (!getCreators(name).output_processor_creator) { @@ -221,7 +221,7 @@ BlockOutputStreamPtr FormatFactory::getOutput( output_getter(buf, sample, std::move(callback), format_settings), sample); } - auto format = getOutputFormat(name, buf, sample, context, std::move(callback)); + auto format = getOutputFormat(name, buf, sample, context, std::move(callback), ignore_no_row_delimiter); return std::make_shared(std::make_shared(format), sample); } @@ -260,7 +260,7 @@ InputFormatPtr FormatFactory::getInputFormat( OutputFormatPtr FormatFactory::getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) @@ -270,6 +270,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( FormatSettings format_settings = getOutputFormatSetting(settings, context); RowOutputFormatParams params; + params.ignore_no_row_delimiter = ignore_no_row_delimiter; params.callback = std::move(callback); /** TODO: Materialization is needed, because formats can use the functions `IDataType`, diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 09fc3cdf7a5..d49414e3944 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -108,7 +108,7 @@ public: ReadCallback callback = {}) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context, WriteCallback callback = {}) const; + const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; InputFormatPtr getInputFormat( const String & name, @@ -119,7 +119,7 @@ public: ReadCallback callback = {}) const; OutputFormatPtr getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback = {}) const; + const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index d12e64050bb..8f28d279c06 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -47,20 +47,20 @@ namespace // SimpleReader is an utility class to deserialize protobufs. // Knows nothing about protobuf schemas, just provides useful functions to deserialize data. -ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_, const bool single_message_mode_) +ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_, const bool use_length_delimiters_) : in(in_) , cursor(0) , current_message_level(0) , current_message_end(0) , field_end(0) , last_string_pos(-1) - , single_message_mode(single_message_mode_) + , use_length_delimiters(use_length_delimiters_) { } [[noreturn]] void ProtobufReader::SimpleReader::throwUnknownFormat() const { - throw Exception(std::string("Protobuf messages are corrupted or don't match the provided schema.") + (single_message_mode ? "" : " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint."), ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); + throw Exception(std::string("Protobuf messages are corrupted or don't match the provided schema.") + (use_length_delimiters ? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint." : ""), ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); } bool ProtobufReader::SimpleReader::startMessage() @@ -70,15 +70,15 @@ bool ProtobufReader::SimpleReader::startMessage() if (unlikely(in.eof())) return false; - if (single_message_mode) - { - current_message_end = END_OF_FILE; - } - else + if (use_length_delimiters) { size_t size_of_message = readVarint(); current_message_end = cursor + size_of_message; } + else + { + current_message_end = END_OF_FILE; + } ++current_message_level; field_end = cursor; return true; @@ -161,16 +161,23 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number) throwUnknownFormat(); } - if (current_message_end == END_OF_FILE) + if (cursor >= current_message_end) { - if (unlikely(in.eof())) + if (current_message_end == END_OF_FILE) { - current_message_end = cursor; - return false; + if (unlikely(in.eof())) + { + current_message_end = cursor; + return false; + } } + else if (current_message_end == END_OF_GROUP) + { + /// We'll check for the `GROUP_END` marker later. + } + else + return false; } - else if ((cursor >= current_message_end) && (current_message_end != END_OF_GROUP)) - return false; UInt64 varint = readVarint(); if (unlikely(varint & (static_cast(0xFFFFFFFF) << 32))) @@ -1096,8 +1103,8 @@ std::unique_ptr ProtobufReader::createConverter & column_names, const bool single_message_mode_) - : simple_reader(in_, single_message_mode_) + ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool use_length_delimiters_) + : simple_reader(in_, use_length_delimiters_) { root_message = ProtobufColumnMatcher::matchColumns(column_names, message_type); setTraitsDataAfterMatchingColumns(root_message.get()); diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index fd9da20781e..b2a0714a57a 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -37,7 +37,7 @@ using AggregateFunctionPtr = std::shared_ptr; class ProtobufReader : private boost::noncopyable { public: - ProtobufReader(ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_); + ProtobufReader(ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool use_length_delimiters_); ~ProtobufReader(); /// Should be called when we start reading a new message. @@ -93,7 +93,7 @@ private: class SimpleReader { public: - SimpleReader(ReadBuffer & in_, const bool single_message_mode_); + SimpleReader(ReadBuffer & in_, const bool use_length_delimiters_); bool startMessage(); void endMessage(bool ignore_errors); void startNestedMessage(); @@ -135,7 +135,7 @@ private: std::vector parent_message_ends; Int64 field_end; Int64 last_string_pos; - const bool single_message_mode; + const bool use_length_delimiters; }; class IConverter diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index 3f9df36329b..e62d8fc4a58 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -21,7 +21,6 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; extern const int PROTOBUF_BAD_CAST; extern const int PROTOBUF_FIELD_NOT_REPEATED; - extern const int TOO_MANY_ROWS; } @@ -124,16 +123,11 @@ namespace // SimpleWriter is an utility class to serialize protobufs. // Knows nothing about protobuf schemas, just provides useful functions to serialize data. -ProtobufWriter::SimpleWriter::SimpleWriter(WriteBuffer & out_, const bool single_message_mode_) +ProtobufWriter::SimpleWriter::SimpleWriter(WriteBuffer & out_, const bool use_length_delimiters_) : out(out_) , current_piece_start(0) , num_bytes_skipped(0) - , produce_length_delimiters(!single_message_mode_) - // normally for single_message_mode we forbid outputting more than one row - // (it would lead to malformed protobuf message), but Kafka/Rabbit push every row - // in a separate message, so it's valid case there and should be allowed. - , allow_several_messages(!single_message_mode_ || out.producesIsolatedRows()) - , row_was_send(false) + , use_length_delimiters(use_length_delimiters_) { } @@ -141,17 +135,12 @@ ProtobufWriter::SimpleWriter::~SimpleWriter() = default; void ProtobufWriter::SimpleWriter::startMessage() { - if (!allow_several_messages && row_was_send) - { - throw Exception("ProtobufSingle can output only single row at a time.", ErrorCodes::TOO_MANY_ROWS); - } - row_was_send = true; } void ProtobufWriter::SimpleWriter::endMessage() { pieces.emplace_back(current_piece_start, buffer.size()); - if (produce_length_delimiters) + if (use_length_delimiters) { size_t size_of_message = buffer.size() - num_bytes_skipped; writeVarint(size_of_message, out); @@ -845,8 +834,8 @@ std::unique_ptr ProtobufWriter::createConverter & column_names, const bool single_message_mode_) - : simple_writer(out, single_message_mode_) + WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool use_length_delimiters_) + : simple_writer(out, use_length_delimiters_) { std::vector field_descriptors_without_match; root_message = ProtobufColumnMatcher::matchColumns(column_names, message_type, field_descriptors_without_match); diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index c49fe6e8edf..44d3aac221e 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -37,7 +37,7 @@ using ConstAggregateDataPtr = const char *; class ProtobufWriter : private boost::noncopyable { public: - ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool single_message_mode_); + ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector & column_names, const bool use_length_delimiters_); ~ProtobufWriter(); /// Should be called at the beginning of writing a message. @@ -89,7 +89,7 @@ private: class SimpleWriter { public: - SimpleWriter(WriteBuffer & out_, const bool single_message_mode_); + SimpleWriter(WriteBuffer & out_, const bool use_length_delimiters_); ~SimpleWriter(); void startMessage(); @@ -138,10 +138,7 @@ private: size_t current_piece_start; size_t num_bytes_skipped; std::vector nested_infos; - const bool produce_length_delimiters; - const bool allow_several_messages; - bool row_was_send; - + const bool use_length_delimiters; }; class IConverter diff --git a/src/Formats/tests/tab_separated_streams.cpp b/src/Formats/tests/tab_separated_streams.cpp index e39c7b6c920..bb38d51cf98 100644 --- a/src/Formats/tests/tab_separated_streams.cpp +++ b/src/Formats/tests/tab_separated_streams.cpp @@ -38,13 +38,14 @@ try FormatSettings format_settings; - RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, []{}}; + RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, []{}}; + RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){},false}; - InputFormatPtr input_format = std::make_shared(sample, in_buf, params, false, false, format_settings); + InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, false, false, format_settings); BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); BlockOutputStreamPtr block_output = std::make_shared( - std::make_shared(out_buf, sample, false, false, [](const Columns & /* columns */, size_t /* row */){}, format_settings)); + std::make_shared(out_buf, sample, false, false, out_params, format_settings)); copyData(*block_input, *block_output); return 0; diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index 6de7534eeff..4fdb814849e 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -60,12 +60,6 @@ public: */ virtual ~WriteBuffer() {} - /** - * some buffers (kafka / rabbit) split the rows internally, so we can push there formats without - * framing / delimiters (like ProtobufSingle) - */ - virtual bool producesIsolatedRows() { return false; } - inline void nextIfAtEnd() { if (!hasPendingData()) diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 73a2cb271fa..4312691ea5e 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -9,13 +9,20 @@ namespace DB { -/// Common parameters for generating blocks. struct RowOutputFormatParams { using WriteCallback = std::function; // Callback used to indicate that another row is written. WriteCallback callback; + + /** + * some buffers (kafka / rabbit) split the rows internally using callback + * so we can push there formats without framing / delimiters + * (like ProtobufSingle). In other cases you can't write more than single row + * in unframed format. + */ + bool ignore_no_row_delimiter = false; }; class WriteBuffer; @@ -26,6 +33,7 @@ class IRowOutputFormat : public IOutputFormat { protected: DataTypes types; + bool first_row = true; void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override; @@ -66,7 +74,6 @@ public: virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes. private: - bool first_row = true; bool prefix_written = false; bool suffix_written = false; diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index ff37efc378e..d1420d0d38e 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -11,10 +11,10 @@ namespace DB { -ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool single_message_mode_) +ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool use_length_delimiters_) : IRowInputFormat(header_, in_, params_) , data_types(header_.getDataTypes()) - , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames(), single_message_mode_) + , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames(), use_length_delimiters_) { } @@ -67,9 +67,9 @@ void ProtobufRowInputFormat::syncAfterError() void registerInputFormatProcessorProtobuf(FormatFactory & factory) { - for (bool single_message_mode : {false, true}) + for (bool use_length_delimiters : {false, true}) { - factory.registerInputFormatProcessor(single_message_mode ? "ProtobufSingle" : "Protobuf", [single_message_mode]( + factory.registerInputFormatProcessor(use_length_delimiters ? "Protobuf" : "ProtobufSingle", [use_length_delimiters]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -78,7 +78,7 @@ void registerInputFormatProcessorProtobuf(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, settings.schema.is_server, settings.schema.format_schema_path), - single_message_mode); + use_length_delimiters); }); } } diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 7f6a7ead0c2..c6bc350e893 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -18,8 +18,8 @@ class FormatSchemaInfo; /** Stream designed to deserialize data from the google protobuf format. * One Protobuf message is parsed as one row of data. * - * Input buffer may contain single protobuf message (single_message_mode_ = true), - * or any number of messages (single_message_mode_ = false). In the second case + * Input buffer may contain single protobuf message (use_length_delimiters_ = false), + * or any number of messages (use_length_delimiters = true). In the second case * parser assumes messages are length-delimited according to documentation * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h * Parsing of the protobuf format requires the 'format_schema' setting to be set, e.g. @@ -29,7 +29,7 @@ class FormatSchemaInfo; class ProtobufRowInputFormat : public IRowInputFormat { public: - ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool single_message_mode_); + ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_, const bool use_length_delimiters_); ~ProtobufRowInputFormat() override; String getName() const override { return "ProtobufRowInputFormat"; } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index e8b7ac0fc93..de213b3a2ea 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -14,6 +14,7 @@ namespace DB { namespace ErrorCodes { + extern const int TOO_MANY_ROWS; } @@ -22,16 +23,22 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( const Block & header, const RowOutputFormatParams & params, const FormatSchemaInfo & format_schema, - const bool single_message_mode_) + const bool use_length_delimiters_) : IRowOutputFormat(header, out_, params) , data_types(header.getDataTypes()) - , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), single_message_mode_) + , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), use_length_delimiters_) + , throw_on_multiple_rows_undelimited(!use_length_delimiters_ && !params.ignore_no_row_delimiter) { value_indices.resize(header.columns()); } void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) { + if (throw_on_multiple_rows_undelimited && !first_row) + { + throw Exception("The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.", ErrorCodes::TOO_MANY_ROWS); + } + writer.startMessage(); std::fill(value_indices.begin(), value_indices.end(), 0); size_t column_index; @@ -44,11 +51,11 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { - for (bool single_message_mode : {false, true}) + for (bool use_length_delimiters : {false, true}) { factory.registerOutputFormatProcessor( - single_message_mode ? "ProtobufSingle" : "Protobuf", - [single_message_mode](WriteBuffer & buf, + use_length_delimiters ? "Protobuf" : "ProtobufSingle", + [use_length_delimiters](WriteBuffer & buf, const Block & header, const RowOutputFormatParams & params, const FormatSettings & settings) @@ -56,7 +63,7 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory) return std::make_shared(buf, header, params, FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, settings.schema.is_server, settings.schema.format_schema_path), - single_message_mode); + use_length_delimiters); }); } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 1183813d26d..97f398b082b 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -26,7 +26,7 @@ namespace DB /** Stream designed to serialize data in the google protobuf format. * Each row is written as a separated message. * - * With single_message_mode=1 it can write only single row as plain protobuf message, + * With use_length_delimiters=0 it can write only single row as plain protobuf message, * otherwise Protobuf messages are delimited according to documentation * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h * Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g. @@ -41,7 +41,7 @@ public: const Block & header, const RowOutputFormatParams & params, const FormatSchemaInfo & format_schema, - const bool single_message_mode_); + const bool use_length_delimiters_); String getName() const override { return "ProtobufRowOutputFormat"; } @@ -53,6 +53,7 @@ private: DataTypes data_types; ProtobufWriter writer; std::vector value_indices; + const bool throw_on_multiple_rows_undelimited; }; } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp index 786edce5edc..90afb4ced30 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -9,8 +9,8 @@ namespace DB RawBLOBRowOutputFormat::RawBLOBRowOutputFormat( WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback) - : IRowOutputFormat(header_, out_, callback) + const RowOutputFormatParams & params) + : IRowOutputFormat(header_, out_, params) { } @@ -27,10 +27,10 @@ void registerOutputFormatProcessorRawBLOB(FormatFactory & factory) factory.registerOutputFormatProcessor("RawBLOB", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback callback, + const RowOutputFormatParams & params, const FormatSettings &) { - return std::make_shared(buf, sample, callback); + return std::make_shared(buf, sample, params); }); } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h index b6c13597d4b..13b526ed8be 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -30,7 +30,7 @@ public: RawBLOBRowOutputFormat( WriteBuffer & out_, const Block & header_, - FormatFactory::WriteCallback callback); + const RowOutputFormatParams & params); String getName() const override { return "RawBLOBRowOutputFormat"; } diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 451c86f1e8e..35ba098b735 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -32,7 +32,7 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row){ buffer->countRow(columns, row); }); + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row){ buffer->countRow(columns, row); }, true); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/src/Storages/Kafka/WriteBufferToKafkaProducer.h index 547d796b576..76859c4e33f 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.h +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.h @@ -25,8 +25,6 @@ public: const Block & header); ~WriteBufferToKafkaProducer() override; - bool producesIsolatedRows() override { return true; } - void countRow(const Columns & columns, size_t row); void flush(); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 8e05b10fa47..778e737e695 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -46,7 +46,9 @@ void RabbitMQBlockOutputStream::writePrefix() storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & /* columns */, size_t /* rows */) { buffer->countRow(); - }); + }, + true + ); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 50ec4b518e1..a8e94070dbd 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -34,7 +34,6 @@ public: ); ~WriteBufferToRabbitMQProducer() override; - bool producesIsolatedRows() override { return true; } void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } From 7fb329713a6619aaf2e8efa68733ea723c499217 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 7 Oct 2020 20:51:10 +0200 Subject: [PATCH 06/10] Avoid shadowing class member, set first_row flag after write --- src/Processors/Formats/IRowOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/AvroRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/AvroRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/BinaryRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/CSVRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/CSVRowOutputFormat.h | 2 +- .../Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp | 4 ++-- .../Formats/Impl/JSONCompactEachRowRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/JSONRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/JSONRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/MarkdownRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/MsgPackRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp | 6 +++--- src/Processors/Formats/Impl/ProtobufRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/TSKVRowOutputFormat.h | 2 +- .../Formats/Impl/TabSeparatedRawRowOutputFormat.h | 4 ++-- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/ValuesRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/VerticalRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/XMLRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/XMLRowOutputFormat.h | 2 +- 34 files changed, 52 insertions(+), 52 deletions(-) diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index 82d83a88cf1..b684dd82261 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -21,9 +21,9 @@ void IRowOutputFormat::consume(DB::Chunk chunk) { if (!first_row) writeRowBetweenDelimiter(); - first_row = false; write(columns, row); + first_row = false; if (params.callback) params.callback(columns, row); diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 74b9541d59a..81922bdde80 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -347,8 +347,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 RowOutputFormatParams & params_, const FormatSettings & settings_) + : IRowOutputFormat(header_, out_, params_) , settings(settings_) , serializer(header_.getColumnsWithTypeAndName()) , file_writer( diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index 2f1953149e1..08370154d9a 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -43,7 +43,7 @@ private: class AvroRowOutputFormat : public IRowOutputFormat { public: - AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & settings_); + AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_); virtual ~AvroRowOutputFormat() override; String getName() const override { return "AvroRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index ad734c5cfa9..d74a0a075fe 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -9,8 +9,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_, const RowOutputFormatParams & params_) + : IRowOutputFormat(header, out_, params_), with_names(with_names_), with_types(with_types_) { } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index e8467663038..562ed7b18aa 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class BinaryRowOutputFormat: 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_, const RowOutputFormatParams & params_); String getName() const override { return "BinaryRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 695ae148749..2d6a49ccb6f 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params), with_names(with_names_), format_settings(format_settings_) +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), with_names(with_names_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 3da76f4fee5..55803aeb53e 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_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); + CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "CSVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index a855c7d2215..e12ca966a93 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -10,11 +10,11 @@ namespace DB JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, bool yield_strings_) - : IRowOutputFormat(header_, out_, params), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) + : IRowOutputFormat(header_, out_, params_), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index 5449168a1ec..eb426bec15d 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -18,7 +18,7 @@ public: JSONCompactEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index b9d4a43d36a..97304afbebd 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -10,10 +10,10 @@ namespace DB JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_) - : JSONRowOutputFormat(out_, header, params, settings_, yield_strings_) + : JSONRowOutputFormat(out_, header, params_, settings_, yield_strings_) { } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index a800537877b..71ba3579837 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -19,7 +19,7 @@ public: JSONCompactRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 5cc14851b28..b3b83949642 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -11,10 +11,10 @@ namespace DB JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_) - : IRowOutputFormat(header_, out_, params), settings(settings_), yield_strings(yield_strings_) + : IRowOutputFormat(header_, out_, params_), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 60cdbcd921e..bd9cfff68c5 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -18,7 +18,7 @@ public: JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index c73abeaa98a..517f126060f 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -10,10 +10,10 @@ namespace DB JSONRowOutputFormat::JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_) - : IRowOutputFormat(header, out_, params), settings(settings_), yield_strings(yield_strings_) + : IRowOutputFormat(header, out_, params_), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 2861fccdac3..88b74afbabd 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -19,7 +19,7 @@ public: JSONRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & settings_, bool yield_strings_); diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index 1ca40522048..51bba07d995 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 RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) {} void MarkdownRowOutputFormat::writePrefix() { diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h index 0b79e8c42ff..6bfb763d818 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -12,7 +12,7 @@ class ReadBuffer; class MarkdownRowOutputFormat : public IRowOutputFormat { public: - MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); + MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); /// Write higher part of markdown table like this: /// |columnName1|columnName2|...|columnNameN| diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index d42c4f7b002..bb20d3d9899 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -24,8 +24,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params) - : IRowOutputFormat(header_, out_, params), packer(out_) {} +MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_) + : IRowOutputFormat(header_, out_, params_), packer(out_) {} void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num) { diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h index 0f91e174367..b6764ed4a4f 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h @@ -20,7 +20,7 @@ namespace DB class MsgPackRowOutputFormat : public IRowOutputFormat { public: - MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params); + MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_); String getName() const override { return "MsgPackRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index de213b3a2ea..9a4e7a1fbdb 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -21,13 +21,13 @@ namespace ErrorCodes ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, const bool use_length_delimiters_) - : IRowOutputFormat(header, out_, params) + : IRowOutputFormat(header, out_, params_) , data_types(header.getDataTypes()) , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), use_length_delimiters_) - , throw_on_multiple_rows_undelimited(!use_length_delimiters_ && !params.ignore_no_row_delimiter) + , throw_on_multiple_rows_undelimited(!use_length_delimiters_ && !params_.ignore_no_row_delimiter) { value_indices.resize(header.columns()); } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 97f398b082b..740efcfa24c 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -39,7 +39,7 @@ public: ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, const bool use_length_delimiters_); diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp index 90afb4ced30..bcee94d8ad5 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -9,8 +9,8 @@ namespace DB RawBLOBRowOutputFormat::RawBLOBRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params) - : IRowOutputFormat(header_, out_, params) + const RowOutputFormatParams & params_) + : IRowOutputFormat(header_, out_, params_) { } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h index 13b526ed8be..6a9a70bb12f 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -30,7 +30,7 @@ public: RawBLOBRowOutputFormat( WriteBuffer & out_, const Block & header_, - const RowOutputFormatParams & params); + const RowOutputFormatParams & params_); String getName() const override { return "RawBLOBRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index 129cb27ce53..d65ce95313e 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, params, format_settings_) +TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out_, header, false, false, params_, format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 98a5846218c..1b341cbbc72 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -14,7 +14,7 @@ namespace DB class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat { public: - TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params, const FormatSettings & format_settings); + TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings); String getName() const override { return "TSKVRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h index b8b0dbc0aa6..6aa7f7bdfad 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h @@ -18,9 +18,9 @@ public: const Block & header_, bool with_names_, bool with_types_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, params, format_settings_) + : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, params_, format_settings_) { } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index f5325b3e869..da8221b11c5 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -11,9 +11,9 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( const Block & header_, bool with_names_, bool with_types_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) + : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 3caf1c7003e..7985d6a1c86 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -23,7 +23,7 @@ public: const Block & header_, bool with_names_, bool with_types_, - const RowOutputFormatParams & params, + const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "TabSeparatedRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index 71dbe4da00c..7791e1296e0 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 RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) { } diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h index 183fd44cb60..73f91866f43 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -15,7 +15,7 @@ class WriteBuffer; class ValuesRowOutputFormat : public IRowOutputFormat { public: - ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); + ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, 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 c6dad489059..a3c71cbde59 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 RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index 36483002325..d372f5f611a 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -18,7 +18,7 @@ class Context; class VerticalRowOutputFormat : public IRowOutputFormat { public: - VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); + VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "VerticalRowOutputFormat"; } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index e36af1610de..a677d0de9a0 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_) - : IRowOutputFormat(header_, out_, params), format_settings(format_settings_) +XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index 91be4db82f9..233ee773c1c 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -16,7 +16,7 @@ namespace DB class XMLRowOutputFormat : public IRowOutputFormat { public: - XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params, const FormatSettings & format_settings_); + XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "XMLRowOutputFormat"; } From 87a8ba63f77190229cbb5e82c95b7e84085ee25d Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 12 Oct 2020 15:05:40 +0200 Subject: [PATCH 07/10] Few more CR fixes --- src/Common/ErrorCodes.cpp | 1 + src/Processors/Formats/IRowOutputFormat.cpp | 3 ++- src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp | 4 ++-- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 8 +++++++- src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 +- tests/queries/0_stateless/00825_protobuf_format_output.sh | 4 ++-- 6 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index bf475bc9b21..1cca3abc996 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -507,6 +507,7 @@ namespace ErrorCodes extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING = 541; extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE = 542; extern const int UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL = 543; + extern const int NO_ROW_DELIMITER = 544; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index b684dd82261..f5f01643f4e 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -23,10 +23,11 @@ void IRowOutputFormat::consume(DB::Chunk chunk) writeRowBetweenDelimiter(); write(columns, row); - first_row = false; if (params.callback) params.callback(columns, row); + + first_row = false; } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 9a4e7a1fbdb..930a83c52da 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -14,7 +14,7 @@ namespace DB { namespace ErrorCodes { - extern const int TOO_MANY_ROWS; + extern const int NO_ROW_DELIMITER; } @@ -36,7 +36,7 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) { if (throw_on_multiple_rows_undelimited && !first_row) { - throw Exception("The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.", ErrorCodes::TOO_MANY_ROWS); + throw Exception("The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.", ErrorCodes::NO_ROW_DELIMITER); } writer.startMessage(); diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 35ba098b735..9d7fe465d44 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -32,7 +32,13 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row){ buffer->countRow(columns, row); }, true); + child = FormatFactory::instance().getOutput( + storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row) + { + buffer->countRow(columns, row); + }, + /* ignore_no_row_delimiter = */ true + ); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 778e737e695..76129dee30d 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -47,7 +47,7 @@ void RabbitMQBlockOutputStream::writePrefix() { buffer->countRow(); }, - true + /* ignore_no_row_delimiter = */ true ); } diff --git a/tests/queries/0_stateless/00825_protobuf_format_output.sh b/tests/queries/0_stateless/00825_protobuf_format_output.sh index 7f0be8b6e84..695f1a0eb38 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -70,8 +70,8 @@ SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETT SELECT 'SQUARES->'; SELECT * FROM out_squares_00825 ORDER BY number LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; --- Code: 158, e.displayText() = DB::Exception: ProtobufSingle can output only single row at a time. -SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 158 } +-- Code: 544, e.displayText() = DB::Exception: The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. +SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 544 } DROP TABLE IF EXISTS out_persons_00825; DROP TABLE IF EXISTS out_squares_00825; From 3e001e7ef1799b4829f3bca854b2b60b9cdddd97 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 13 Oct 2020 15:19:38 +0300 Subject: [PATCH 08/10] Update Settings.h --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a7a7bd6d4b7..17c01270683 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -403,7 +403,7 @@ class IColumn; M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) // End of COMMON_SETTINGS -// !!! please add settings related to formats into the FORMAT_FACTORY_SETTINGS below !!! +// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. #define FORMAT_FACTORY_SETTINGS(M) \ M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ @@ -470,7 +470,7 @@ class IColumn; M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) // End of FORMAT_FACTORY_SETTINGS -// !!! please add settings non-related to formats into the COMMON_SETTINGS above !!! +// Please add settings non-related to formats into the COMMON_SETTINGS above. #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ From fc285bf82cecd68a1d784ed53e6446c4bf437d4e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 14 Oct 2020 16:30:15 +0300 Subject: [PATCH 09/10] Update 00825_protobuf_format_output.sh Fix the error code after resolving conflict --- .../0_stateless/00825_protobuf_format_output.sh | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00825_protobuf_format_output.sh b/tests/queries/0_stateless/00825_protobuf_format_output.sh index 695f1a0eb38..5c8b0fdedab 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -19,7 +19,13 @@ CREATE TABLE out_persons_00825 (uuid UUID, surname String, gender Enum8('male'=1, 'female'=0), birthDate Date, - photo Nullable(String), + photo Nullable(String),<<<<<<< non_delimited_protobuf +511 +  +======= +512 +  + phoneNumber Nullable(FixedString(13)), isOnline UInt8, visitTime Nullable(DateTime), @@ -70,8 +76,8 @@ SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETT SELECT 'SQUARES->'; SELECT * FROM out_squares_00825 ORDER BY number LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; --- Code: 544, e.displayText() = DB::Exception: The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. -SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 544 } +-- Code: 546, e.displayText() = DB::Exception: The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. +SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 546 } DROP TABLE IF EXISTS out_persons_00825; DROP TABLE IF EXISTS out_squares_00825; From a33d84f3a47dee884895d43987fdccd1baec14f3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 14 Oct 2020 16:32:33 +0300 Subject: [PATCH 10/10] Update 00825_protobuf_format_output.sh --- tests/queries/0_stateless/00825_protobuf_format_output.sh | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00825_protobuf_format_output.sh b/tests/queries/0_stateless/00825_protobuf_format_output.sh index 5c8b0fdedab..889d3a9d2ae 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -19,13 +19,7 @@ CREATE TABLE out_persons_00825 (uuid UUID, surname String, gender Enum8('male'=1, 'female'=0), birthDate Date, - photo Nullable(String),<<<<<<< non_delimited_protobuf -511 -  -======= -512 -  - + photo Nullable(String), phoneNumber Nullable(FixedString(13)), isOnline UInt8, visitTime Nullable(DateTime),