diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 1bbb194df23..b34131acfb0 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -108,6 +108,7 @@ add_headers_and_sources(dbms src/Formats) add_headers_and_sources(dbms src/Processors) add_headers_and_sources(dbms src/Processors/Executors) add_headers_and_sources(dbms src/Processors/Formats) +add_headers_and_sources(dbms src/Processors/Formats/Impl) add_headers_only(dbms src/Server) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 9b97d8464ee..166a58cf214 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -26,15 +27,16 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT); } - -BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf, const Block & sample, const Context & context, UInt64 max_block_size) const +const FormatFactory::ProcessorCreators & FormatFactory::getProcessorCreators(const String & name) const { - const auto & input_getter = getCreators(name).first; - if (!input_getter) - throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - - const Settings & settings = context.getSettingsRef(); + auto it = processors_dict.find(name); + if (processors_dict.end() != it) + return it->second; + throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT); +} +static FormatSettings getInputFormatSetting(const Settings & settings) +{ FormatSettings format_settings; format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; @@ -46,18 +48,11 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; - return input_getter(buf, sample, context, max_block_size, format_settings); + return format_settings; } - -BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const +static FormatSettings getOutputFormatSetting(const Settings & settings) { - const auto & output_getter = getCreators(name).second; - if (!output_getter) - throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; @@ -70,6 +65,32 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & format_settings.pretty.color = settings.output_format_pretty_color; format_settings.write_statistics = settings.output_format_write_statistics; + return format_settings; +} + + +BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf, const Block & sample, const Context & context, UInt64 max_block_size) const +{ + const auto & input_getter = getCreators(name).first; + if (!input_getter) + throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); + + const Settings & settings = context.getSettingsRef(); + FormatSettings format_settings = getInputFormatSetting(settings); + + return input_getter(buf, sample, context, max_block_size, format_settings); +} + + +BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const +{ + const auto & output_getter = getCreators(name).second; + if (!output_getter) + throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); + + const Settings & settings = context.getSettingsRef(); + FormatSettings format_settings = getOutputFormatSetting(settings); + /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ @@ -78,12 +99,46 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & } +InputFormatPtr FormatFactory::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, const Context & context, UInt64 max_block_size) const +{ + const auto & input_getter = getProcessorCreators(name).first; + if (!input_getter) + throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); + + const Settings & settings = context.getSettingsRef(); + FormatSettings format_settings = getInputFormatSetting(settings); + + RowInputFormatParams params; + params.max_block_size = max_block_size; + params.allow_errors_num = format_settings.input_allow_errors_num; + params.allow_errors_ratio = format_settings.input_allow_errors_ratio; + + return input_getter(buf, sample, context, params, format_settings); +} + + +OutputFormatPtr FormatFactory::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const +{ + const auto & output_getter = getProcessorCreators(name).second; + if (!output_getter) + throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); + + const Settings & settings = context.getSettingsRef(); + FormatSettings format_settings = getOutputFormatSetting(settings); + + /** TODO: Materialization is needed, because formats can use the functions `IDataType`, + * which only work with full columns. + */ + return output_getter(buf, sample, context, format_settings); +} + + void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) { auto & target = dict[name].first; if (target) throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = input_creator; + target = std::move(input_creator); } void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator) @@ -91,7 +146,23 @@ void FormatFactory::registerOutputFormat(const String & name, OutputCreator outp auto & target = dict[name].second; if (target) throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = output_creator; + target = std::move(output_creator); +} + +void FormatFactory::registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator) +{ + auto & target = processors_dict[name].first; + if (target) + throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = std::move(input_creator); +} + +void FormatFactory::registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator) +{ + auto & target = processors_dict[name].second; + if (target) + throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = std::move(output_creator); } @@ -113,6 +184,22 @@ void registerInputFormatJSONEachRow(FormatFactory & factory); void registerOutputFormatJSONEachRow(FormatFactory & factory); void registerOutputFormatProtobuf(FormatFactory & factory); +void registerInputFormatProcessorNative(FormatFactory & factory); +void registerOutputFormatProcessorNative(FormatFactory & factory); +void registerInputFormatProcessorRowBinary(FormatFactory & factory); +void registerOutputFormatProcessorRowBinary(FormatFactory & factory); +void registerInputFormatProcessorTabSeparated(FormatFactory & factory); +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); +void registerInputFormatProcessorValues(FormatFactory & factory); +void registerOutputFormatProcessorValues(FormatFactory & factory); +void registerInputFormatProcessorCSV(FormatFactory & factory); +void registerOutputFormatProcessorCSV(FormatFactory & factory); +void registerInputFormatProcessorTSKV(FormatFactory & factory); +void registerOutputFormatProcessorTSKV(FormatFactory & factory); +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorProtobuf(FormatFactory & factory); + /// Output only (presentational) formats. void registerOutputFormatPretty(FormatFactory & factory); @@ -126,9 +213,21 @@ void registerOutputFormatODBCDriver(FormatFactory & factory); void registerOutputFormatODBCDriver2(FormatFactory & factory); void registerOutputFormatNull(FormatFactory & factory); +void registerOutputFormatProcessorPretty(FormatFactory & factory); +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); +void registerOutputFormatProcessorVertical(FormatFactory & factory); +void registerOutputFormatProcessorJSON(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorXML(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); +void registerOutputFormatProcessorNull(FormatFactory & factory); + /// Input only formats. void registerInputFormatCapnProto(FormatFactory & factory); +void registerInputFormatProcessorCapnProto(FormatFactory & factory); FormatFactory::FormatFactory() @@ -150,6 +249,23 @@ FormatFactory::FormatFactory() registerOutputFormatProtobuf(*this); registerInputFormatCapnProto(*this); + registerInputFormatProcessorNative(*this); + registerOutputFormatProcessorNative(*this); + registerInputFormatProcessorRowBinary(*this); + registerOutputFormatProcessorRowBinary(*this); + registerInputFormatProcessorTabSeparated(*this); + registerOutputFormatProcessorTabSeparated(*this); + registerInputFormatProcessorValues(*this); + registerOutputFormatProcessorValues(*this); + registerInputFormatProcessorCSV(*this); + registerOutputFormatProcessorCSV(*this); + registerInputFormatProcessorTSKV(*this); + registerOutputFormatProcessorTSKV(*this); + registerInputFormatProcessorJSONEachRow(*this); + registerOutputFormatProcessorJSONEachRow(*this); + registerOutputFormatProcessorProtobuf(*this); + registerInputFormatProcessorCapnProto(*this); + registerOutputFormatPretty(*this); registerOutputFormatPrettyCompact(*this); registerOutputFormatPrettySpace(*this); @@ -160,6 +276,17 @@ FormatFactory::FormatFactory() registerOutputFormatODBCDriver(*this); registerOutputFormatODBCDriver2(*this); registerOutputFormatNull(*this); + + registerOutputFormatProcessorPretty(*this); + registerOutputFormatProcessorPrettyCompact(*this); + registerOutputFormatProcessorPrettySpace(*this); + registerOutputFormatProcessorVertical(*this); + registerOutputFormatProcessorJSON(*this); + registerOutputFormatProcessorJSONCompact(*this); + registerOutputFormatProcessorXML(*this); + registerOutputFormatProcessorODBCDriver(*this); + registerOutputFormatProcessorODBCDriver2(*this); + registerOutputFormatProcessorNull(*this); } } diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index fed069bbf8a..65ae5e2fc7a 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -23,6 +23,14 @@ class IBlockOutputStream; using BlockInputStreamPtr = std::shared_ptr; using BlockOutputStreamPtr = std::shared_ptr; +class IInputFormat; +class IOutputFormat; + +class RowInputFormatParams; + +using InputFormatPtr = std::shared_ptr; +using OutputFormatPtr = std::shared_ptr; + /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. @@ -43,9 +51,24 @@ private: const Context & context, const FormatSettings & settings)>; + using InputProcessorCreator = std::function; + + using OutputProcessorCreator = std::function; + using Creators = std::pair; + using ProcessorCreators = std::pair; using FormatsDictionary = std::unordered_map; + using FormatProcessorsDictionary = std::unordered_map; public: BlockInputStreamPtr getInput(const String & name, ReadBuffer & buf, @@ -54,10 +77,19 @@ public: BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const; + InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, + const Block & sample, const Context & context, UInt64 max_block_size) const; + + OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, + const Block & sample, const Context & context) const; + /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); void registerOutputFormat(const String & name, OutputCreator output_creator); + void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator); + void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator); + const FormatsDictionary & getAllFormats() const { return dict; @@ -65,11 +97,13 @@ public: private: FormatsDictionary dict; + FormatProcessorsDictionary processors_dict; FormatFactory(); friend class ext::singleton; const Creators & getCreators(const String & name) const; + const ProcessorCreators & getProcessorCreators(const String & name) const; }; } diff --git a/dbms/src/Processors/Chunk.cpp b/dbms/src/Processors/Chunk.cpp new file mode 100644 index 00000000000..feec58b98d7 --- /dev/null +++ b/dbms/src/Processors/Chunk.cpp @@ -0,0 +1,120 @@ +#include +#include + +namespace DB +{ + +Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_) +{ + checkNumRowsIsConsistent(); +} + +Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) + : columns(std::move(columns_)), num_rows(num_rows_), chunk_info(std::move(chunk_info_)) +{ + checkNumRowsIsConsistent(); +} + +static Columns unmuteColumns(MutableColumns && mut_columns) +{ + Columns columns; + columns.reserve(mut_columns.size()); + for (auto & col : mut_columns) + columns.emplace_back(std::move(col)); + + return columns; +} + +Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_) + : columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_) +{ +} + +Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) + : columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_), chunk_info(std::move(chunk_info_)) +{ +} + +Chunk::Chunk(Chunk && other) noexcept + : columns(std::move(other.columns)) + , num_rows(other.num_rows) + , chunk_info(std::move(other.chunk_info)) +{ + other.num_rows = 0; +} + +Chunk & Chunk::operator=(Chunk && other) noexcept +{ + columns = std::move(other.columns); + chunk_info = std::move(other.chunk_info); + num_rows = other.num_rows; + other.num_rows = 0; + return *this; +} + +void Chunk::setColumns(Columns columns_, UInt64 num_rows_) +{ + columns = std::move(columns_); + num_rows = num_rows_; + checkNumRowsIsConsistent(); +} + +void Chunk::setColumns(MutableColumns columns_, UInt64 num_rows_) +{ + columns = unmuteColumns(std::move(columns_)); + num_rows = num_rows_; + checkNumRowsIsConsistent(); +} + +void Chunk::checkNumRowsIsConsistent() +{ + for (auto & column : columns) + if (column->size() != num_rows) + throw Exception("Invalid number of rows in Chunk column " + column->getName()+ ": expected " + + toString(num_rows) + ", got " + toString(column->size()), ErrorCodes::LOGICAL_ERROR); +} + +MutableColumns Chunk::mutateColumns() +{ + size_t num_columns = columns.size(); + MutableColumns mut_columns(num_columns); + for (size_t i = 0; i < num_columns; ++i) + mut_columns[i] = (*std::move(columns[i])).mutate(); + + columns.clear(); + num_rows = 0; + + return mut_columns; +} + +Columns Chunk::detachColumns() +{ + num_rows = 0; + return std::move(columns); +} + +void Chunk::clear() +{ + num_rows = 0; + columns.clear(); + chunk_info.reset(); +} + + +void ChunkMissingValues::setBit(size_t column_idx, size_t row_idx) +{ + RowsBitMask & mask = rows_mask_by_column_id[column_idx]; + mask.resize(row_idx + 1); + mask[row_idx] = true; +} + +const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(size_t column_idx) const +{ + static RowsBitMask none; + auto it = rows_mask_by_column_id.find(column_idx); + if (it != rows_mask_by_column_id.end()) + return it->second; + return none; +} + +} diff --git a/dbms/src/Processors/Chunk.h b/dbms/src/Processors/Chunk.h new file mode 100644 index 00000000000..121fd46a73e --- /dev/null +++ b/dbms/src/Processors/Chunk.h @@ -0,0 +1,76 @@ +#pragma once + +#include + +namespace DB +{ + +class ChunkInfo +{ +public: + virtual ~ChunkInfo() = default; +}; + +using ChunkInfoPtr = std::shared_ptr; + +class Chunk +{ +public: + Chunk() = default; + Chunk(const Chunk & other) = default; + Chunk(Chunk && other) noexcept; + Chunk(Columns columns_, UInt64 num_rows_); + Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); + Chunk(MutableColumns columns_, UInt64 num_rows_); + Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); + + Chunk & operator=(const Chunk & other) = default; + Chunk & operator=(Chunk && other) noexcept; + + const Columns & getColumns() const { return columns; } + void setColumns(Columns columns_, UInt64 num_rows_); + void setColumns(MutableColumns columns_, UInt64 num_rows_); + Columns detachColumns(); + MutableColumns mutateColumns(); + + const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } + void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } + + UInt64 getNumRows() const { return num_rows; } + UInt64 getNumColumns() const { return columns.size(); } + bool empty() const { return num_rows == 0; } + operator bool() const { return !empty() || !columns.empty(); } + + void clear(); + +private: + Columns columns; + UInt64 num_rows = 0; + ChunkInfoPtr chunk_info; + + void checkNumRowsIsConsistent(); +}; + +using Chunks = std::vector; + +/// Block extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. +class ChunkMissingValues : public ChunkInfo +{ +public: + using RowsBitMask = std::vector; /// a bit per row for a column + + const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; + void setBit(size_t column_idx, size_t row_idx); + bool empty() const { return rows_mask_by_column_id.empty(); } + size_t size() const { return rows_mask_by_column_id.size(); } + void clear() { rows_mask_by_column_id.clear(); } + +private: + using RowsMaskByColumnId = std::unordered_map; + + /// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default. + /// It could contain less columns and rows then related block. + RowsMaskByColumnId rows_mask_by_column_id; +}; + +} diff --git a/dbms/src/Processors/Formats/IInputFormat.h b/dbms/src/Processors/Formats/IInputFormat.h index 8cfec18b4f9..7e4e00c1a33 100644 --- a/dbms/src/Processors/Formats/IInputFormat.h +++ b/dbms/src/Processors/Formats/IInputFormat.h @@ -12,7 +12,7 @@ class ReadBuffer; */ class IInputFormat : public ISource { -private: +protected: /// Skip GCC warning: ‘maybe_unused’ attribute ignored #pragma GCC diagnostic push diff --git a/dbms/src/Processors/Formats/IOutputFormat.cpp b/dbms/src/Processors/Formats/IOutputFormat.cpp index cf882d96230..0083c15b457 100644 --- a/dbms/src/Processors/Formats/IOutputFormat.cpp +++ b/dbms/src/Processors/Formats/IOutputFormat.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB @@ -35,11 +36,23 @@ IOutputFormat::Status IOutputFormat::prepare() return Status::Ready; } + finished = true; + + if (!finalized) + return Status::Ready; + return Status::Finished; } void IOutputFormat::work() { + if (finished && !finalized) + { + finalize(); + finalized = true; + return; + } + switch (current_block_kind) { case Main: @@ -56,5 +69,10 @@ void IOutputFormat::work() has_input = false; } +void IOutputFormat::flush() +{ + out.next(); +} + } diff --git a/dbms/src/Processors/Formats/IOutputFormat.h b/dbms/src/Processors/Formats/IOutputFormat.h index c7fe5d038c7..6a12309f56a 100644 --- a/dbms/src/Processors/Formats/IOutputFormat.h +++ b/dbms/src/Processors/Formats/IOutputFormat.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -27,12 +28,15 @@ protected: WriteBuffer & out; Chunk current_chunk; - PortKind current_block_kind; + PortKind current_block_kind = PortKind::Main; bool has_input = false; + bool finished = false; + bool finalized = false; virtual void consume(Chunk) = 0; virtual void consumeTotals(Chunk) {} virtual void consumeExtremes(Chunk) {} + virtual void finalize() {} public: IOutputFormat(Block header, WriteBuffer & out); @@ -40,14 +44,19 @@ public: Status prepare() override; void work() override; - void flush(); + /// Flush output buffers if any. + virtual void flush(); - /** Content-Type to set when sending HTTP response. - */ + /// Value for rows_before_limit_at_least field. + virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} + + /// Notify about progress. Method could be called from different threads. + /// Passed value are delta, that must be summarized. + virtual void onProgress(const Progress & /*progress*/) {} + + /// Content-Type to set when sending HTTP response. virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; } - /// TODO onProgress, rows_before_limit_at_least - InputPort & getPort(PortKind kind) { return inputs[kind]; } }; } diff --git a/dbms/src/Processors/Formats/IRowInputFormat.cpp b/dbms/src/Processors/Formats/IRowInputFormat.cpp index 0318f2dcfb7..bbd9479bbc9 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowInputFormat.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_NUMBER; extern const int CANNOT_PARSE_UUID; extern const int TOO_LARGE_STRING_SIZE; + extern const int INCORRECT_NUMBER_OF_COLUMNS; } @@ -42,6 +43,8 @@ Chunk IRowInputFormat::generate() MutableColumns columns = header.cloneEmptyColumns(); size_t prev_rows = total_rows; + ChunkMissingValues chunk_missing_values; + try { for (size_t rows = 0; rows < params.max_block_size; ++rows) @@ -49,8 +52,21 @@ Chunk IRowInputFormat::generate() try { ++total_rows; - if (!readRow(columns)) + + RowReadExtension info; + if (!readRow(columns, info)) break; + + for (size_t column_idx = 0; column_idx < info.read_columns.size(); ++column_idx) + { + if (!info.read_columns[column_idx]) + { + size_t column_size = columns[column_idx]->size(); + if (column_size == 0) + throw Exception("Unexpected empty column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS); + chunk_missing_values.setBit(column_idx, column_size - 1); + } + } } catch (Exception & e) { @@ -122,9 +138,14 @@ Chunk IRowInputFormat::generate() return {}; } - Chunk chunk; - chunk.setColumns(std::move(columns), total_rows - prev_rows); + Chunk chunk(std::move(columns), total_rows - prev_rows); + chunk.setChunkInfo(std::make_unique(std::move(chunk_missing_values))); return chunk; } +void IRowInputFormat::syncAfterError() +{ + throw Exception("Method syncAfterError is not implemented for input format", ErrorCodes::NOT_IMPLEMENTED); +} + } diff --git a/dbms/src/Processors/Formats/IRowInputFormat.h b/dbms/src/Processors/Formats/IRowInputFormat.h index 59b178c4c78..732568460ac 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.h +++ b/dbms/src/Processors/Formats/IRowInputFormat.h @@ -8,25 +8,34 @@ namespace DB { -/** Row oriented input format: reads data row by row. - */ +/// Contains extra information about read data. +struct RowReadExtension +{ + /// IRowInputStream.read() output. It contains non zero for columns that actually read from the source and zero otherwise. + /// It's used to attach defaults for partially filled rows. + std::vector read_columns; +}; + +/// Common parameters for generating blocks. +struct RowInputFormatParams +{ + size_t max_block_size; + + UInt64 allow_errors_num; + Float64 allow_errors_ratio; +}; + +///Row oriented input format: reads data row by row. class IRowInputFormat : public IInputFormat { public: - /// Common parameters for generating blocks. - struct Params - { - size_t max_block_size; - - UInt64 allow_errors_num; - Float64 allow_errors_ratio; - }; + using Params = RowInputFormatParams; IRowInputFormat( Block header, ReadBuffer & in, Params params) - : IInputFormat(header, in), params(std::move(params)) + : IInputFormat(std::move(header), in), params(params) { } @@ -36,7 +45,7 @@ protected: /** Read next row and append it to the columns. * If no more rows - return false. */ - virtual bool readRow(MutableColumns & columns) = 0; + virtual bool readRow(MutableColumns & columns, RowReadExtension & extra) = 0; virtual void readPrefix() {}; /// delimiter before begin of result virtual void readSuffix() {}; /// delimiter after end of result diff --git a/dbms/src/Processors/Formats/IRowOutputFormat.cpp b/dbms/src/Processors/Formats/IRowOutputFormat.cpp index ee5cc18e985..6db2802f232 100644 --- a/dbms/src/Processors/Formats/IRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowOutputFormat.cpp @@ -1,10 +1,68 @@ #include #include +#include namespace DB { +void IRowOutputFormat::consume(DB::Chunk chunk) +{ + writePrefixIfNot(); + + auto num_rows = chunk.getNumRows(); + auto & columns = chunk.getColumns(); + + for (UInt64 row = 0; row < num_rows; ++row) + { + if (!first_row) + writeRowBetweenDelimiter(); + first_row = false; + + write(columns, row); + } +} + +void IRowOutputFormat::consumeTotals(DB::Chunk chunk) +{ + writePrefixIfNot(); + writeSuffixIfNot(); + + auto num_rows = chunk.getNumRows(); + if (num_rows != 1) + throw Exception("Got " + toString(num_rows) + " in totals chunk, expected 1", ErrorCodes::LOGICAL_ERROR); + + auto & columns = chunk.getColumns(); + + writeBeforeTotals(); + writeTotals(columns, 0); + writeAfterTotals(); +} + +void IRowOutputFormat::consumeExtremes(DB::Chunk chunk) +{ + writePrefixIfNot(); + writeSuffixIfNot(); + + auto num_rows = chunk.getNumRows(); + auto & columns = chunk.getColumns(); + if (num_rows != 2) + throw Exception("Got " + toString(num_rows) + " in extremes chunk, expected 2", ErrorCodes::LOGICAL_ERROR); + + writeBeforeExtremes(); + writeMinExtreme(columns, 0); + writeRowBetweenDelimiter(); + writeMaxExtreme(columns, 1); + writeAfterExtremes(); +} + +void IRowOutputFormat::finalize() +{ + writePrefixIfNot(); + writeSuffixIfNot(); + writeLastSuffix(); +} + void IRowOutputFormat::write(const Columns & columns, size_t row_num) { size_t num_columns = columns.size(); @@ -22,6 +80,21 @@ void IRowOutputFormat::write(const Columns & columns, size_t row_num) writeRowEndDelimiter(); } +void IRowOutputFormat::writeMinExtreme(const DB::Columns & columns, size_t row_num) +{ + write(columns, row_num); +} + +void IRowOutputFormat::writeMaxExtreme(const DB::Columns & columns, size_t row_num) +{ + write(columns, row_num); +} + +void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num) +{ + write(columns, row_num); +} + } diff --git a/dbms/src/Processors/Formats/IRowOutputFormat.h b/dbms/src/Processors/Formats/IRowOutputFormat.h index 14b9a56ba0a..c70c7cb75c5 100644 --- a/dbms/src/Processors/Formats/IRowOutputFormat.h +++ b/dbms/src/Processors/Formats/IRowOutputFormat.h @@ -19,6 +19,7 @@ protected: void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override; void consumeExtremes(Chunk chunk) override; + void finalize() override; public: IRowOutputFormat(Block header, WriteBuffer & out) @@ -31,21 +32,47 @@ public: * (except delimiter between rows (writeRowBetweenDelimiter())). */ virtual void write(const Columns & columns, size_t row_num); + virtual void writeMinExtreme(const Columns & columns, size_t row_num); + virtual void writeMaxExtreme(const Columns & columns, size_t row_num); + virtual void writeTotals(const Columns & columns, size_t row_num); /** Write single value. */ virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num) = 0; /** Write delimiter. */ - virtual void writeFieldDelimiter() {}; /// delimiter between values - virtual void writeRowStartDelimiter() {}; /// delimiter before each row - virtual void writeRowEndDelimiter() {}; /// delimiter after each row - virtual void writeRowBetweenDelimiter() {}; /// delimiter between rows - virtual void writePrefix() {}; /// delimiter before resultset - virtual void writeSuffix() {}; /// delimiter after resultset - virtual void writeBeforeTotals() {}; - virtual void writeAfterTotals() {}; - virtual void writeBeforeExtremes() {}; - virtual void writeAfterExtremes() {}; + virtual void writeFieldDelimiter() {} /// delimiter between values + virtual void writeRowStartDelimiter() {} /// delimiter before each row + virtual void writeRowEndDelimiter() {} /// delimiter after each row + virtual void writeRowBetweenDelimiter() {} /// delimiter between rows + virtual void writePrefix() {} /// delimiter before resultset + virtual void writeSuffix() {} /// delimiter after resultset + virtual void writeBeforeTotals() {} + virtual void writeAfterTotals() {} + virtual void writeBeforeExtremes() {} + virtual void writeAfterExtremes() {} + virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes. + +private: + bool first_row = true; + bool prefix_written = false; + bool suffix_written = false; + + void writePrefixIfNot() + { + if (!prefix_written) + writePrefix(); + + prefix_written = true; + } + + void writeSuffixIfNot() + { + if (!suffix_written) + writeSuffix(); + + suffix_written = true; + } + }; } diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp new file mode 100644 index 00000000000..6267dd15da6 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include + + +namespace DB +{ + +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in, Block header, Params params, bool with_names_, bool with_types_) + : IRowInputFormat(std::move(header), in, params), with_names(with_names_), with_types(with_types_) +{ +} + + +bool BinaryRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + for (size_t i = 0; i < num_columns; ++i) + getPort().getHeader().getByPosition(i).type->deserializeBinary(*columns[i], in); + + return true; +} + + +void BinaryRowInputFormat::readPrefix() +{ + /// NOTE The header is completely ignored. This can be easily improved. + + UInt64 columns = 0; + String tmp; + + if (with_names || with_types) + { + readVarUInt(columns, in); + } + + if (with_names) + { + for (size_t i = 0; i < columns; ++i) + { + readStringBinary(tmp, in); + } + } + + if (with_types) + { + for (size_t i = 0; i < columns; ++i) + { + readStringBinary(tmp, in); + } + } +} + + +void registerInputFormatProcessorRowBinary(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("RowBinary", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + const IRowInputFormat::Params & params, + const FormatSettings &) + { + return std::make_shared(buf, sample, params, false, false); + }); + + factory.registerInputFormatProcessor("RowBinaryWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + const IRowInputFormat::Params & params, + const FormatSettings &) + { + return std::make_shared(buf, sample, params, true, true); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.h new file mode 100644 index 00000000000..5abc48e4060 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ReadBuffer; + + +/** A stream for inputting data in a binary line-by-line format. + */ +class BinaryRowInputFormat : public IRowInputFormat +{ +public: + BinaryRowInputFormat(ReadBuffer & in, Block header, Params params, bool with_names_, bool with_types_); + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + void readPrefix() override; + + String getName() const override { return "BinaryRowInputFormat"; } + +private: + bool with_names; + bool with_types; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp new file mode 100644 index 00000000000..a39adc7f451 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out, Block header, bool with_names_, bool with_types_) + : IRowOutputFormat(std::move(header), out), with_names(with_names_), with_types(with_types_) +{ +} + +void BinaryRowOutputFormat::writePrefix() +{ + auto & header = getPort(PortKind::Main).getHeader(); + size_t columns = header.columns(); + + if (with_names || with_types) + { + writeVarUInt(columns, out); + } + + if (with_names) + { + for (size_t i = 0; i < columns; ++i) + { + writeStringBinary(header.safeGetByPosition(i).name, out); + } + } + + if (with_types) + { + for (size_t i = 0; i < columns; ++i) + { + writeStringBinary(header.safeGetByPosition(i).type->getName(), out); + } + } +} + +void BinaryRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeBinary(column, row_num, out); +} + + +void registerOutputFormatProcessorRowBinary(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("RowBinary", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings &) + { + return std::make_shared(buf, sample, false, false); + }); + + factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings &) + { + return std::make_shared(buf, sample, true, true); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h new file mode 100644 index 00000000000..626553a0eb3 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class IColumn; +class IDataType; +class WriteBuffer; + + +/** A stream for outputting data in a binary line-by-line format. + */ +class BinaryRowOutputFormat: public IRowOutputFormat +{ +public: + BinaryRowOutputFormat(WriteBuffer & out, Block header, bool with_names_, bool with_types_); + + String getName() const override { return "BinaryRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writePrefix() override; + + void flush() override; + + String getContentType() const override { return "application/octet-stream"; } + +protected: + bool with_names; + bool with_types; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/CMakeLists.txt b/dbms/src/Processors/Formats/Impl/CMakeLists.txt new file mode 100644 index 00000000000..65172356645 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CMakeLists.txt @@ -0,0 +1,3 @@ +if (ENABLE_TESTS) + add_subdirectory (tests) +endif () diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp new file mode 100644 index 00000000000..52194c010e7 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -0,0 +1,367 @@ +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; +} + + +CSVRowInputFormat::CSVRowInputFormat( + ReadBuffer & in, Block header, Params params, bool with_names_, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header), in, params), with_names(with_names_), format_settings(format_settings) +{ + auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + data_types.resize(num_columns); + for (size_t i = 0; i < num_columns; ++i) + data_types[i] = sample.safeGetByPosition(i).type; +} + + +static void skipEndOfLine(ReadBuffer & istr) +{ + /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) + + if (*istr.position() == '\n') + { + ++istr.position(); + if (!istr.eof() && *istr.position() == '\r') + ++istr.position(); + } + else if (*istr.position() == '\r') + { + ++istr.position(); + if (!istr.eof() && *istr.position() == '\n') + ++istr.position(); + else + throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)." + " Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA); + } + else if (!istr.eof()) + throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA); +} + + +static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column) +{ + if (is_last_column) + { + if (istr.eof()) + return; + + /// we support the extra delimiter at the end of the line + if (*istr.position() == delimiter) + { + ++istr.position(); + if (istr.eof()) + return; + } + + skipEndOfLine(istr); + } + else + assertChar(delimiter, istr); +} + + +/// Skip `whitespace` symbols allowed in CSV. +static inline void skipWhitespacesAndTabs(ReadBuffer & buf) +{ + while (!buf.eof() + && (*buf.position() == ' ' + || *buf.position() == '\t')) + ++buf.position(); +} + + +static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns) +{ + String tmp; + for (size_t i = 0; i < num_columns; ++i) + { + skipWhitespacesAndTabs(istr); + readCSVString(tmp, istr, settings); + skipWhitespacesAndTabs(istr); + + skipDelimiter(istr, settings.delimiter, i + 1 == num_columns); + } +} + + +void CSVRowInputFormat::readPrefix() +{ + /// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes, + /// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it. + skipBOMIfExists(in); + + size_t num_columns = data_types.size(); + String tmp; + + if (with_names) + skipRow(in, format_settings.csv, num_columns); +} + + +bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + if (in.eof()) + return false; + + updateDiagnosticInfo(); + + size_t size = data_types.size(); + + for (size_t i = 0; i < size; ++i) + { + skipWhitespacesAndTabs(in); + data_types[i]->deserializeAsTextCSV(*columns[i], in, format_settings); + skipWhitespacesAndTabs(in); + + skipDelimiter(in, format_settings.csv.delimiter, i + 1 == size); + } + + return true; +} + + +String CSVRowInputFormat::getDiagnosticInfo() +{ + if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed. + return {}; + + WriteBufferFromOwnString out; + + auto & header = getPort().getHeader(); + MutableColumns columns = header.cloneEmptyColumns(); + + /// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer. + size_t bytes_read_at_start_of_buffer = in.count() - in.offset(); + if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) + { + out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; + return out.str(); + } + + size_t max_length_of_column_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) + max_length_of_column_name = header.safeGetByPosition(i).name.size(); + + size_t max_length_of_data_type_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) + max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); + + /// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information. + + if (pos_of_prev_row) + { + in.position() = pos_of_prev_row; + + out << "\nRow " << (row_num - 1) << ":\n"; + if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) + return out.str(); + } + else + { + if (!pos_of_current_row) + { + out << "Could not print diagnostic info because parsing of data hasn't started.\n"; + return out.str(); + } + + in.position() = pos_of_current_row; + } + + out << "\nRow " << row_num << ":\n"; + parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); + out << "\n"; + + return out.str(); +} + + +bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, + WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) +{ + const char delimiter = format_settings.csv.delimiter; + auto & header = getPort().getHeader(); + + size_t size = data_types.size(); + for (size_t i = 0; i < size; ++i) + { + if (i == 0 && in.eof()) + { + out << "\n"; + return false; + } + + out << "Column " << i << ", " << std::string((i < 10 ? 2 : i < 100 ? 1 : 0), ' ') + << "name: " << header.safeGetByPosition(i).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(i).name.size(), ' ') + << "type: " << data_types[i]->getName() << ", " << std::string(max_length_of_data_type_name - data_types[i]->getName().size(), ' '); + + BufferBase::Position prev_position = in.position(); + BufferBase::Position curr_position = in.position(); + std::exception_ptr exception; + + try + { + skipWhitespacesAndTabs(in); + prev_position = in.position(); + data_types[i]->deserializeAsTextCSV(*columns[i], in, format_settings); + curr_position = in.position(); + skipWhitespacesAndTabs(in); + } + catch (...) + { + exception = std::current_exception(); + } + + if (curr_position < prev_position) + throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); + + if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i])) + { + /// An empty string instead of a value. + if (curr_position == prev_position) + { + out << "ERROR: text "; + verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out); + out << " is not like " << data_types[i]->getName() << "\n"; + return false; + } + } + + out << "parsed text: "; + verbosePrintString(prev_position, curr_position, out); + + if (exception) + { + if (data_types[i]->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (data_types[i]->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + else + out << "ERROR\n"; + return false; + } + + out << "\n"; + + if (data_types[i]->haveMaximumSizeOfValue()) + { + if (*curr_position != '\n' && *curr_position != '\r' && *curr_position != delimiter) + { + out << "ERROR: garbage after " << data_types[i]->getName() << ": "; + verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out); + out << "\n"; + + if (data_types[i]->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (data_types[i]->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + + return false; + } + } + + /// Delimiters + if (i + 1 == size) + { + if (in.eof()) + return false; + + /// we support the extra delimiter at the end of the line + if (*in.position() == delimiter) + { + ++in.position(); + if (in.eof()) + break; + } + + if (!in.eof() && *in.position() != '\n' && *in.position() != '\r') + { + out << "ERROR: There is no line feed. "; + verbosePrintString(in.position(), in.position() + 1, out); + out << " found instead.\n" + " It's like your file has more columns than expected.\n" + "And if your file have right number of columns, maybe it have unquoted string value with comma.\n"; + + return false; + } + + skipEndOfLine(in); + } + else + { + try + { + assertChar(delimiter, in); + } + catch (const DB::Exception &) + { + if (*in.position() == '\n' || *in.position() == '\r') + { + out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected." + " It's like your file has less columns than expected.\n" + "And if your file have right number of columns, maybe it have unescaped quotes in values.\n"; + } + else + { + out << "ERROR: There is no delimiter (" << delimiter << "). "; + verbosePrintString(in.position(), in.position() + 1, out); + out << " found instead.\n"; + } + return false; + } + } + } + + return true; +} + + +void CSVRowInputFormat::syncAfterError() +{ + skipToNextLineOrEOF(in); +} + +void CSVRowInputFormat::updateDiagnosticInfo() +{ + ++row_num; + + bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; + bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset(); + + pos_of_prev_row = pos_of_current_row; + pos_of_current_row = in.position(); +} + + +void registerInputFormatProcessorCSV(FormatFactory & factory) +{ + for (bool with_names : {false, true}) + { + factory.registerInputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, with_names, settings); + }); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h new file mode 100644 index 00000000000..b861f064d3d --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class ReadBuffer; + +/** A stream for inputting data in csv format. + * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. + */ +class CSVRowInputFormat : public IRowInputFormat +{ +public: + /** with_names - in the first line the header with column names + * with_types - on the next line header with type names + */ + CSVRowInputFormat(ReadBuffer & in, Block header, Params params, bool with_names, const FormatSettings & format_settings); + + String getName() const override { return "CSVRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + void readPrefix() override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + + std::string getDiagnosticInfo() override; + +private: + bool with_names; + DataTypes data_types; + + const FormatSettings format_settings; + + /// For convenient diagnostics in case of an error. + + size_t row_num = 0; + + /// How many bytes were read, not counting those that are still in the buffer. + size_t bytes_read_at_start_of_buffer_on_current_row = 0; + size_t bytes_read_at_start_of_buffer_on_prev_row = 0; + + char * pos_of_current_row = nullptr; + char * pos_of_prev_row = nullptr; + + void updateDiagnosticInfo(); + + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, + WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp new file mode 100644 index 00000000000..3a431777219 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -0,0 +1,87 @@ +#include +#include + +#include + + +namespace DB +{ + + +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out, Block header, bool with_names_, const FormatSettings & format_settings) + : IRowOutputFormat(std::move(header), out), with_names(with_names_), format_settings(format_settings) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + size_t columns = sample.columns(); + data_types.resize(columns); + for (size_t i = 0; i < columns; ++i) + data_types[i] = sample.safeGetByPosition(i).type; +} + + +void CSVRowOutputFormat::writePrefix() +{ + auto & sample = getPort(PortKind::Main).getHeader(); + size_t columns = sample.columns(); + + if (with_names) + { + for (size_t i = 0; i < columns; ++i) + { + writeCSVString(sample.safeGetByPosition(i).name, out); + + char delimiter = format_settings.csv.delimiter; + if (i + 1 == columns) + delimiter = '\n'; + + writeChar(delimiter, out); + } + } +} + + +void CSVRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextCSV(column, row_num, out, format_settings); +} + + +void CSVRowOutputFormat::writeFieldDelimiter() +{ + writeChar(format_settings.csv.delimiter, out); +} + + +void CSVRowOutputFormat::writeRowEndDelimiter() +{ + writeChar('\n', out); +} + +void CSVRowOutputFormat::writeBeforeTotals() +{ + writeChar('\n', out); +} + +void CSVRowOutputFormat::writeBeforeExtremes() +{ + writeChar('\n', out); +} + + + +void registerOutputFormatProcessorCSV(FormatFactory & factory) +{ + for (bool with_names : {false, true}) + { + factory.registerOutputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, with_names, format_settings); + }); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h new file mode 100644 index 00000000000..861d0f2b5fd --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class WriteBuffer; + + +/** The stream for outputting data in csv format. + * Does not conform with https://tools.ietf.org/html/rfc4180 because it uses LF, not CR LF. + */ +class CSVRowOutputFormat : public IRowOutputFormat +{ +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, Block header, bool with_names_, const FormatSettings & format_settings); + + String getName() const override { return "CSVRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowEndDelimiter() override; + void writePrefix() override; + void writeBeforeTotals() override; + void writeBeforeExtremes() override; + + /// https://www.iana.org/assignments/media-types/text/csv + String getContentType() const override + { + return String("text/csv; charset=UTF-8; header=") + (with_names ? "present" : "absent"); + } + +protected: + + bool with_names; + const FormatSettings format_settings; + DataTypes data_types; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp new file mode 100644 index 00000000000..8466cd99a5a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -0,0 +1,314 @@ +#include +#if USE_CAPNP + +#include +#include +#include // Y_IGNORE +#include +#include +#include // Y_IGNORE +#include // Y_IGNORE +#include // Y_IGNORE +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_TYPE_OF_FIELD; + extern const int BAD_ARGUMENTS; + extern const int THERE_IS_NO_COLUMN; + extern const int LOGICAL_ERROR; +} + +CapnProtoRowInputFormat::NestedField split(const Block & header, size_t i) +{ + CapnProtoRowInputFormat::NestedField field = {{}, i}; + + // Remove leading dot in field definition, e.g. ".msg" -> "msg" + String name(header.safeGetByPosition(i).name); + if (!name.empty() && name[0] == '.') + name.erase(0, 1); + + boost::split(field.tokens, name, boost::is_any_of("._")); + return field; +} + + +Field convertNodeToField(const capnp::DynamicValue::Reader & value) +{ + switch (value.getType()) + { + case capnp::DynamicValue::UNKNOWN: + throw Exception("Unknown field type", ErrorCodes::BAD_TYPE_OF_FIELD); + case capnp::DynamicValue::VOID: + return Field(); + case capnp::DynamicValue::BOOL: + return value.as() ? 1u : 0u; + case capnp::DynamicValue::INT: + return value.as(); + case capnp::DynamicValue::UINT: + return value.as(); + case capnp::DynamicValue::FLOAT: + return value.as(); + case capnp::DynamicValue::TEXT: + { + auto arr = value.as(); + return String(arr.begin(), arr.size()); + } + case capnp::DynamicValue::DATA: + { + auto arr = value.as().asChars(); + return String(arr.begin(), arr.size()); + } + case capnp::DynamicValue::LIST: + { + auto listValue = value.as(); + Array res(listValue.size()); + for (auto i : kj::indices(listValue)) + res[i] = convertNodeToField(listValue[i]); + + return res; + } + case capnp::DynamicValue::ENUM: + return value.as().getRaw(); + case capnp::DynamicValue::STRUCT: + { + auto structValue = value.as(); + const auto & fields = structValue.getSchema().getFields(); + + Field field = Tuple(TupleBackend(fields.size())); + TupleBackend & tuple = get(field).toUnderType(); + for (auto i : kj::indices(fields)) + tuple[i] = convertNodeToField(structValue.get(fields[i])); + + return field; + } + case capnp::DynamicValue::CAPABILITY: + throw Exception("CAPABILITY type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); + case capnp::DynamicValue::ANY_POINTER: + throw Exception("ANY_POINTER type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); + } + return Field(); +} + +capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std::string & field) +{ + KJ_IF_MAYBE(child, node.findFieldByName(field)) + return *child; + else + throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr(), ErrorCodes::THERE_IS_NO_COLUMN); +} + + +void CapnProtoRowInputFormat::createActions(const NestedFieldList & sorted_fields, capnp::StructSchema reader) +{ + /// Columns in a table can map to fields in Cap'n'Proto or to structs. + + /// Store common parents and their tokens in order to backtrack. + std::vector parents; + std::vector parent_tokens; + + capnp::StructSchema cur_reader = reader; + + for (const auto & field : sorted_fields) + { + if (field.tokens.empty()) + throw Exception("Logical error in CapnProtoRowInputFormat", ErrorCodes::LOGICAL_ERROR); + + // Backtrack to common parent + while (field.tokens.size() < parent_tokens.size() + 1 + || !std::equal(parent_tokens.begin(), parent_tokens.end(), field.tokens.begin())) + { + actions.push_back({Action::POP}); + parents.pop_back(); + parent_tokens.pop_back(); + + if (parents.empty()) + { + cur_reader = reader; + break; + } + else + cur_reader = parents.back().getType().asStruct(); + } + + // Go forward + while (parent_tokens.size() + 1 < field.tokens.size()) + { + const auto & token = field.tokens[parents.size()]; + auto node = getFieldOrThrow(cur_reader, token); + if (node.getType().isStruct()) + { + // Descend to field structure + parents.emplace_back(node); + parent_tokens.emplace_back(token); + cur_reader = node.getType().asStruct(); + actions.push_back({Action::PUSH, node}); + } + else if (node.getType().isList()) + { + break; // Collect list + } + else + throw Exception("Field " + token + " is neither Struct nor List", ErrorCodes::BAD_TYPE_OF_FIELD); + } + + // Read field from the structure + auto node = getFieldOrThrow(cur_reader, field.tokens[parents.size()]); + if (node.getType().isList() && !actions.empty() && actions.back().field == node) + { + // The field list here flattens Nested elements into multiple arrays + // In order to map Nested types in Cap'nProto back, they need to be collected + // Since the field names are sorted, the order of field positions must be preserved + // For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first + // even though it's position is second. + auto & columns = actions.back().columns; + auto it = std::upper_bound(columns.cbegin(), columns.cend(), field.pos); + columns.insert(it, field.pos); + } + else + { + actions.push_back({Action::READ, node, {field.pos}}); + } + } +} + +CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in, Block header, Params params, const FormatSchemaInfo & info) + : IRowInputFormat(std::move(header), in, params), parser(std::make_shared()) +{ + // Parse the schema and fetch the root object + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" + auto schema = parser->impl.parseDiskFile(info.schemaPath(), info.absoluteSchemaPath(), {}); +#pragma GCC diagnostic pop + + root = schema.getNested(info.messageName()).asStruct(); + + /** + * The schema typically consists of fields in various nested structures. + * Here we gather the list of fields and sort them in a way so that fields in the same structure are adjacent, + * and the nesting level doesn't decrease to make traversal easier. + */ + auto & sample = getPort().getHeader(); + NestedFieldList list; + size_t num_columns = sample.columns(); + for (size_t i = 0; i < num_columns; ++i) + list.push_back(split(sample, i)); + + // Order list first by value of strings then by length of string vector. + std::sort(list.begin(), list.end(), [](const NestedField & a, const NestedField & b) { return a.tokens < b.tokens; }); + createActions(list, root); +} + + +bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + if (in.eof()) + return false; + + // Read from underlying buffer directly + auto buf = in.buffer(); + auto base = reinterpret_cast(in.position()); + + // Check if there's enough bytes in the buffer to read the full message + kj::Array heap_array; + auto array = kj::arrayPtr(base, buf.size() - in.offset()); + auto expected_words = capnp::expectedSizeInWordsFromPrefix(array); + if (expected_words * sizeof(capnp::word) > array.size()) + { + // We'll need to reassemble the message in a contiguous buffer + heap_array = kj::heapArray(expected_words); + in.readStrict(heap_array.asChars().begin(), heap_array.asChars().size()); + array = heap_array.asPtr(); + } + + +#if CAPNP_VERSION >= 8000 + capnp::UnalignedFlatArrayMessageReader msg(array); +#else + capnp::FlatArrayMessageReader msg(array); +#endif + std::vector stack; + stack.push_back(msg.getRoot(root)); + + for (auto action : actions) + { + switch (action.type) + { + case Action::READ: + { + Field value = convertNodeToField(stack.back().get(action.field)); + if (action.columns.size() > 1) + { + // Nested columns must be flattened into several arrays + // e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...) + const auto & collected = DB::get(value); + size_t size = collected.size(); + // The flattened array contains an array of a part of the nested tuple + Array flattened(size); + for (size_t column_index = 0; column_index < action.columns.size(); ++column_index) + { + // Populate array with a single tuple elements + for (size_t off = 0; off < size; ++off) + { + const TupleBackend & tuple = DB::get(collected[off]).toUnderType(); + flattened[off] = tuple[column_index]; + } + auto & col = columns[action.columns[column_index]]; + col->insert(flattened); + } + } + else + { + auto & col = columns[action.columns[0]]; + col->insert(value); + } + + break; + } + case Action::POP: + stack.pop_back(); + break; + case Action::PUSH: + stack.push_back(stack.back().get(action.field).as()); + break; + } + } + + // Advance buffer position if used directly + if (heap_array.size() == 0) + { + auto parsed = (msg.getEnd() - base) * sizeof(capnp::word); + in.position() += parsed; + } + + return true; +} + +void registerInputFormatProcessorCapnProto(FormatFactory & factory) +{ + factory.registerInputFormatProcessor( + "CapnProto", + [](ReadBuffer & buf, const Block & sample, const Context & context, IRowInputFormat::Params params, const FormatSettings &) + { + return std::make_shared(buf, sample, params, FormatSchemaInfo(context, "capnp")); + }); +} + +} + +#else + +namespace DB +{ + class FormatFactory; + void registerInputFormatCapnProto(FormatFactory &) {} +} + +#endif // USE_CAPNP diff --git a/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h new file mode 100644 index 00000000000..6824a4013d6 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -0,0 +1,75 @@ +#pragma once +#include +#if USE_CAPNP + +#include +#include + +#include + +namespace DB +{ + +class FormatSchemaInfo; +class ReadBuffer; + +/** A stream for reading messages in Cap'n Proto format in given schema. + * Like Protocol Buffers and Thrift (but unlike JSON or MessagePack), + * Cap'n Proto messages are strongly-typed and not self-describing. + * The schema in this case cannot be compiled in, so it uses a runtime schema parser. + * See https://capnproto.org/cxx.html + */ +class CapnProtoRowInputFormat : public IRowInputFormat +{ +public: + struct NestedField + { + std::vector tokens; + size_t pos; + }; + using NestedFieldList = std::vector; + + /** schema_dir - base path for schema files + * schema_file - location of the capnproto schema, e.g. "schema.capnp" + * root_object - name to the root object, e.g. "Message" + */ + CapnProtoRowInputFormat(ReadBuffer & in, Block header, Params params, const FormatSchemaInfo & info); + + String getName() const override { return "CapnProtoRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + +private: + // Build a traversal plan from a sorted list of fields + void createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader); + + /* Action for state machine for traversing nested structures. */ + using BlockPositionList = std::vector; + struct Action + { + enum Type { POP, PUSH, READ }; + Type type; + capnp::StructSchema::Field field = {}; + BlockPositionList columns = {}; + }; + + // Wrapper for classes that could throw in destructor + // https://github.com/capnproto/capnproto/issues/553 + template + struct DestructorCatcher + { + T impl; + template + DestructorCatcher(Arg && ... args) : impl(kj::fwd(args)...) {} + ~DestructorCatcher() noexcept try { } catch (...) { return; } + }; + using SchemaParser = DestructorCatcher; + + std::shared_ptr parser; + capnp::StructSchema root; + std::vector actions; +}; + +} + +#endif // USE_CAPNP diff --git a/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.cpp b/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.cpp new file mode 100644 index 00000000000..b7a5da7bf04 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.cpp @@ -0,0 +1,81 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schema_file_extension, bool schema_required) +{ + String format_schema = context.getSettingsRef().format_schema.toString(); + if (format_schema.empty()) + { + if (schema_required) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'"), + ErrorCodes::BAD_ARGUMENTS); + } + return; + } + + size_t colon_pos = format_schema.find(':'); + Poco::Path path; + if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1) + || path.assign(format_schema.substr(0, colon_pos)).getFileName().empty()) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'") + ". Got '" + format_schema + + "'", + ErrorCodes::BAD_ARGUMENTS); + } + + is_null = false; + message_name = format_schema.substr(colon_pos + 1); + + auto default_schema_directory = [&context]() + { + static const String str = Poco::Path(context.getFormatSchemaPath()).makeAbsolute().makeDirectory().toString(); + return str; + }; + auto is_server = [&context]() + { + return context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); + }; + + if (path.getExtension().empty() && !schema_file_extension.empty()) + path.setExtension(schema_file_extension); + + if (path.isAbsolute()) + { + if (is_server()) + throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); + schema_path = path.getFileName(); + schema_directory = path.makeParent().toString(); + } + else if (path.depth() >= 1 && path.directory(0) == "..") + { + if (is_server()) + throw Exception( + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), + ErrorCodes::BAD_ARGUMENTS); + path = Poco::Path(default_schema_directory()).resolve(path).toString(); + schema_path = path.getFileName(); + schema_directory = path.makeParent().toString(); + } + else + { + schema_path = path.toString(); + schema_directory = default_schema_directory(); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.h b/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.h new file mode 100644 index 00000000000..6ad9e6fb2b0 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/FormatSchemaInfo.h @@ -0,0 +1,35 @@ +#pragma once + +#include + +namespace DB +{ +class Context; + +/// Extracts information about where the format schema file is from passed context and keep it. +class FormatSchemaInfo +{ +public: + FormatSchemaInfo() = default; + FormatSchemaInfo(const Context & context, const String & schema_file_extension = String(), bool schema_required = true); + + bool isNull() const { return is_null; } + + /// Returns path to the schema file. + const String & schemaPath() const { return schema_path; } + String absoluteSchemaPath() const { return schema_directory + schema_path; } + + /// Returns directory containing the schema file. + const String & schemaDirectory() const { return schema_directory; } + + /// Returns name of the message type. + const String & messageName() const { return message_name; } + +private: + bool is_null = true; + String schema_path; + String schema_directory; + String message_name; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp new file mode 100644 index 00000000000..aae21b93661 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -0,0 +1,86 @@ +#include +#include + +#include + + +namespace DB +{ + +JSONCompactRowOutputFormat::JSONCompactRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings) + : JSONRowOutputFormat(out, std::move(header), settings) +{ +} + + +void JSONCompactRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; +} + + +void JSONCompactRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", *ostr); +} + + +void JSONCompactRowOutputFormat::writeRowStartDelimiter() +{ + if (row_count > 0) + writeCString(",\n", *ostr); + writeCString("\t\t[", *ostr); +} + + +void JSONCompactRowOutputFormat::writeRowEndDelimiter() +{ + writeChar(']', *ostr); + field_number = 0; + ++row_count; +} + +void JSONCompactRowOutputFormat::writeBeforeTotals() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"totals\": [", *ostr); +} + +void JSONCompactRowOutputFormat::writeAfterTotals() +{ + writeChar(']', *ostr); +} + +void JSONCompactRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + writeCString("\t\t\"", *ostr); + writeCString(title, *ostr); + writeCString("\": [", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + if (i != 0) + writeFieldDelimiter(); + + writeField(*columns[i], *types[i], row_num); + } + + writeChar(']', *ostr); +} + +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONCompact", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h new file mode 100644 index 00000000000..94d529ebbb6 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct FormatSettings; + +/** The stream for outputting data in the JSONCompact format. + */ +class JSONCompactRowOutputFormat : public JSONRowOutputFormat +{ +public: + JSONCompactRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings); + + String getName() const override { return "JSONCompactRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + +protected: + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; + + void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override + { + return writeField(column, type, row_num); + } +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp new file mode 100644 index 00000000000..9119216f88f --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -0,0 +1,270 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; + extern const int LOGICAL_ERROR; +} + +namespace +{ + +enum +{ + UNKNOWN_FIELD = size_t(-1), + NESTED_FIELD = size_t(-2) +}; + +} + + +JSONEachRowRowInputFormat::JSONEachRowRowInputFormat( + ReadBuffer & in, Block header, Params params, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header), in, params), format_settings(format_settings), name_map(header.columns()) +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); + + size_t num_columns = getPort().getHeader().columns(); + for (size_t i = 0; i < num_columns; ++i) + { + const String & column_name = columnName(i); + name_map[column_name] = i; /// NOTE You could place names more cache-locally. + if (format_settings.import_nested_json) + { + const auto splitted = Nested::splitName(column_name); + if (!splitted.second.empty()) + { + const StringRef table_name(column_name.data(), splitted.first.size()); + name_map[table_name] = NESTED_FIELD; + } + } + } + + prev_positions.assign(num_columns, name_map.end()); +} + +const String & JSONEachRowRowInputFormat::columnName(size_t i) const +{ + return getPort().getHeader().getByPosition(i).name; +} + +inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index) +{ + /// Optimization by caching the order of fields (which is almost always the same) + /// and a quick check to match the next expected field, instead of searching the hash table. + + if (prev_positions.size() > key_index + && prev_positions[key_index] != name_map.end() + && name == prev_positions[key_index]->first) + { + return prev_positions[key_index]->second; + } + else + { + const auto it = name_map.find(name); + + if (name_map.end() != it) + { + if (key_index < prev_positions.size()) + prev_positions[key_index] = it; + + return it->second; + } + else + return UNKNOWN_FIELD; + } +} + +/** Read the field name and convert it to column name + * (taking into account the current nested name prefix) + * Resulting StringRef is valid only before next read from buf. + */ +StringRef JSONEachRowRowInputFormat::readColumnName(ReadBuffer & buf) +{ + // This is just an optimization: try to avoid copying the name into current_column_name + + if (nested_prefix_length == 0 && buf.position() + 1 < buf.buffer().end()) + { + char * next_pos = find_first_symbols<'\\', '"'>(buf.position() + 1, buf.buffer().end()); + + if (next_pos != buf.buffer().end() && *next_pos != '\\') + { + /// The most likely option is that there is no escape sequence in the key name, and the entire name is placed in the buffer. + assertChar('"', buf); + StringRef res(buf.position(), next_pos - buf.position()); + buf.position() = next_pos + 1; + return res; + } + } + + current_column_name.resize(nested_prefix_length); + readJSONStringInto(current_column_name, buf); + return current_column_name; +} + + +static inline void skipColonDelimeter(ReadBuffer & istr) +{ + skipWhitespaceIfAny(istr); + assertChar(':', istr); + skipWhitespaceIfAny(istr); +} + +void JSONEachRowRowInputFormat::skipUnknownField(const StringRef & name_ref) +{ + if (!format_settings.skip_unknown_fields) + throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); + + skipJSONField(in, name_ref); +} + +void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + if (read_columns[index]) + throw Exception("Duplicate field found while parsing JSONEachRow format: " + columnName(index), ErrorCodes::INCORRECT_DATA); + + try + { + auto & header = getPort().getHeader(); + header.getByPosition(index).type->deserializeAsTextJSON(*columns[index], in, format_settings); + } + catch (Exception & e) + { + e.addMessage("(while read the value of key " + columnName(index) + ")"); + throw; + } + + read_columns[index] = true; +} + +inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index) +{ + skipWhitespaceIfAny(in); + + if (in.eof()) + throw Exception("Unexpected end of stream while parsing JSONEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); + else if (*in.position() == '}') + { + ++in.position(); + return false; + } + + if (key_index > 0) + { + assertChar(',', in); + skipWhitespaceIfAny(in); + } + return true; +} + +void JSONEachRowRowInputFormat::readJSONObject(MutableColumns & columns) +{ + assertChar('{', in); + + for (size_t key_index = 0; advanceToNextKey(key_index); ++key_index) + { + StringRef name_ref = readColumnName(in); + const size_t column_index = columnIndex(name_ref, key_index); + + if (unlikely(ssize_t(column_index) < 0)) + { + /// name_ref may point directly to the input buffer + /// and input buffer may be filled with new data on next read + /// If we want to use name_ref after another reads from buffer, we must copy it to temporary string. + + current_column_name.assign(name_ref.data, name_ref.size); + name_ref = StringRef(current_column_name); + + skipColonDelimeter(in); + + if (column_index == UNKNOWN_FIELD) + skipUnknownField(name_ref); + else if (column_index == NESTED_FIELD) + readNestedData(name_ref.toString(), columns); + else + throw Exception("Logical error: illegal value of column_index", ErrorCodes::LOGICAL_ERROR); + } + else + { + skipColonDelimeter(in); + readField(column_index, columns); + } + } +} + +void JSONEachRowRowInputFormat::readNestedData(const String & name, MutableColumns & columns) +{ + current_column_name = name; + current_column_name.push_back('.'); + nested_prefix_length = current_column_name.size(); + readJSONObject(columns); + nested_prefix_length = 0; +} + + +bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + skipWhitespaceIfAny(in); + + /// We consume ;, or \n before scanning a new row, instead scanning to next row at the end. + /// The reason is that if we want an exact number of rows read with LIMIT x + /// from a streaming table engine with text data format, like File or Kafka + /// then seeking to next ;, or \n would trigger reading of an extra row at the end. + + /// Semicolon is added for convenience as it could be used at end of INSERT query. + if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) + ++in.position(); + + skipWhitespaceIfAny(in); + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + + /// Set of columns for which the values were read. The rest will be filled with default values. + read_columns.assign(num_columns, false); + + nested_prefix_length = 0; + readJSONObject(columns); + + auto & header = getPort().getHeader(); + /// Fill non-visited columns with the default values. + for (size_t i = 0; i < num_columns; ++i) + if (!read_columns[i]) + header.getByPosition(i).type->insertDefaultInto(*columns[i]); + + /// return info about defaults set + ext.read_columns = read_columns; + return true; +} + + +void JSONEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + + +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("JSONEachRow", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h new file mode 100644 index 00000000000..4949cd0cd7a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class ReadBuffer; + + +/** A stream for reading data in JSON format, where each row is represented by a separate JSON object. + * Objects can be separated by feed return, other whitespace characters in any number and possibly a comma. + * Fields can be listed in any order (including, in different lines there may be different order), + * and some fields may be missing. + */ +class JSONEachRowRowInputFormat : public IRowInputFormat +{ +public: + JSONEachRowRowInputFormat(ReadBuffer & in, Block header, Params params, const FormatSettings & format_settings); + + String getName() const override { return "JSONEachRowRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + +private: + const String & columnName(size_t i) const; + size_t columnIndex(const StringRef & name, size_t key_index); + bool advanceToNextKey(size_t key_index); + void skipUnknownField(const StringRef & name_ref); + StringRef readColumnName(ReadBuffer & buf); + void readField(size_t index, MutableColumns & columns); + void readJSONObject(MutableColumns & columns); + void readNestedData(const String & name, MutableColumns & columns); + +private: + + const FormatSettings format_settings; + + /// Buffer for the read from the stream field name. Used when you have to copy it. + /// Also, if processing of Nested data is in progress, it holds the common prefix + /// of the nested column names (so that appending the field name to it produces + /// the full column name) + String current_column_name; + + /// If processing Nested data, holds the length of the common prefix + /// of the names of related nested columns. For example, for a table + /// created as follows + /// CREATE TABLE t (n Nested (i Int32, s String)) + /// the nested column names are 'n.i' and 'n.s' and the nested prefix is 'n.' + size_t nested_prefix_length = 0; + + std::vector read_columns; + + /// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map. + using NameMap = HashMap; + NameMap name_map; + + /// Cached search results for previous row (keyed as index in JSON object) - used as a hint. + std::vector prev_positions; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp new file mode 100644 index 00000000000..434bc3f56b1 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings) + : IRowOutputFormat(std::move(header), out), settings(settings) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + size_t columns = sample.columns(); + fields.resize(columns); + + for (size_t i = 0; i < columns; ++i) + { + WriteBufferFromString buf(fields[i]); + writeJSONString(sample.getByPosition(i).name, buf, settings); + } +} + + +void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + writeString(fields[field_number], out); + writeChar(':', out); + type.serializeAsTextJSON(column, row_num, out, settings); + ++field_number; +} + + +void JSONEachRowRowOutputFormat::writeFieldDelimiter() +{ + writeChar(',', out); +} + + +void JSONEachRowRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('{', out); +} + + +void JSONEachRowRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("}\n", out); + field_number = 0; +} + + +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONEachRow", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h new file mode 100644 index 00000000000..150a9b6ad0e --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line. + * Does not validate UTF-8. + */ +class JSONEachRowRowOutputFormat : public IRowOutputFormat +{ +public: + JSONEachRowRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings); + + String getName() const override { return "JSONEachRowRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + +private: + size_t field_number = 0; + Names fields; + + FormatSettings settings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp new file mode 100644 index 00000000000..3d210d4fc86 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -0,0 +1,257 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings) + : IRowOutputFormat(std::move(header), out), settings(settings) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); + + bool need_validate_utf8 = false; + for (size_t i = 0; i < sample.columns(); ++i) + { + if (!sample.getByPosition(i).type->textCanContainOnlyValidUTF8()) + need_validate_utf8 = true; + + WriteBufferFromOwnString buf; + writeJSONString(fields[i].name, buf, settings); + + fields[i].name = buf.str(); + } + + if (need_validate_utf8) + { + validating_ostr = std::make_unique(out); + ostr = validating_ostr.get(); + } + else + ostr = &out; +} + + +void JSONRowOutputFormat::writePrefix() +{ + writeCString("{\n", *ostr); + writeCString("\t\"meta\":\n", *ostr); + writeCString("\t[\n", *ostr); + + for (size_t i = 0; i < fields.size(); ++i) + { + writeCString("\t\t{\n", *ostr); + + writeCString("\t\t\t\"name\": ", *ostr); + writeString(fields[i].name, *ostr); + writeCString(",\n", *ostr); + writeCString("\t\t\t\"type\": ", *ostr); + writeJSONString(fields[i].type->getName(), *ostr, settings); + writeChar('\n', *ostr); + + writeCString("\t\t}", *ostr); + if (i + 1 < fields.size()) + writeChar(',', *ostr); + writeChar('\n', *ostr); + } + + writeCString("\t],\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"data\":\n", *ostr); + writeCString("\t[\n", *ostr); +} + + +void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + writeCString("\t\t\t", *ostr); + writeString(fields[field_number].name, *ostr); + writeCString(": ", *ostr); + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; +} + +void JSONRowOutputFormat::writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) +{ + writeCString("\t\t", *ostr); + writeJSONString(fields[field_number].name, *ostr, settings); + writeCString(": ", *ostr); + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; +} + +void JSONRowOutputFormat::writeFieldDelimiter() +{ + writeCString(",\n", *ostr); +} + + +void JSONRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("\t\t{\n", *ostr); +} + + +void JSONRowOutputFormat::writeRowEndDelimiter() +{ + writeChar('\n', *ostr); + writeCString("\t\t}", *ostr); + field_number = 0; + ++row_count; +} + + +void JSONRowOutputFormat::writeRowBetweenDelimiter() +{ + writeCString(",\n", *ostr); +} + + +void JSONRowOutputFormat::writeSuffix() +{ + writeChar('\n', *ostr); + writeCString("\t]", *ostr); +} + +void JSONRowOutputFormat::writeBeforeTotals() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"totals\":\n", *ostr); + writeCString("\t{\n", *ostr); +} + +void JSONRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + size_t num_columns = columns.size(); + + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + writeFieldDelimiter(); + + writeTotalsField(*columns[i], *types[i], row_num); + } +} + +void JSONRowOutputFormat::writeAfterTotals() +{ + writeChar('\n', *ostr); + writeCString("\t}", *ostr); + field_number = 0; +} + +void JSONRowOutputFormat::writeBeforeExtremes() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"extremes\":\n", *ostr); + writeCString("\t{\n", *ostr); +} + +void JSONRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + writeCString("\t\t\"", *ostr); + writeCString(title, *ostr); + writeCString("\":\n", *ostr); + writeCString("\t\t{\n", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + if (i != 0) + writeFieldDelimiter(); + + writeField(*columns[i], *types[i], row_num); + } + + writeChar('\n', *ostr); + writeCString("\t\t}", *ostr); + field_number = 0; +} + +void JSONRowOutputFormat::writeMinExtreme(const Columns & columns, size_t row_num) +{ + writeExtremesElement("min", columns, row_num); +} + +void JSONRowOutputFormat::writeMaxExtreme(const Columns & columns, size_t row_num) +{ + writeExtremesElement("max", columns, row_num); +} + +void JSONRowOutputFormat::writeAfterExtremes() +{ + writeChar('\n', *ostr); + writeCString("\t}", *ostr); +} + +void JSONRowOutputFormat::writeLastSuffix() +{ + writeCString(",\n\n", *ostr); + writeCString("\t\"rows\": ", *ostr); + writeIntText(row_count, *ostr); + + writeRowsBeforeLimitAtLeast(); + + if (settings.write_statistics) + writeStatistics(); + + writeChar('\n', *ostr); + writeCString("}\n", *ostr); + ostr->next(); +} + +void JSONRowOutputFormat::writeRowsBeforeLimitAtLeast() +{ + if (applied_limit) + { + writeCString(",\n\n", *ostr); + writeCString("\t\"rows_before_limit_at_least\": ", *ostr); + writeIntText(rows_before_limit, *ostr); + } +} + +void JSONRowOutputFormat::writeStatistics() +{ + writeCString(",\n\n", *ostr); + writeCString("\t\"statistics\":\n", *ostr); + writeCString("\t{\n", *ostr); + + writeCString("\t\t\"elapsed\": ", *ostr); + writeText(watch.elapsedSeconds(), *ostr); + writeCString(",\n", *ostr); + writeCString("\t\t\"rows_read\": ", *ostr); + writeText(progress.rows.load(), *ostr); + writeCString(",\n", *ostr); + writeCString("\t\t\"bytes_read\": ", *ostr); + writeText(progress.bytes.load(), *ostr); + writeChar('\n', *ostr); + + writeCString("\t}", *ostr); +} + +void JSONRowOutputFormat::onProgress(const Progress & value) +{ + progress.incrementPiecewiseAtomically(value); +} + + +void registerOutputFormatProcessorProcessorJSON(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSON", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h new file mode 100644 index 00000000000..81029d42131 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -0,0 +1,83 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Stream for output data in JSON format. + */ +class JSONRowOutputFormat : public IRowOutputFormat +{ +public: + JSONRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & settings); + + String getName() const override { return "JSONRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; + void writePrefix() override; + void writeSuffix() override; + + void writeMinExtreme(const Columns & columns, size_t row_num) override; + void writeMaxExtreme(const Columns & columns, size_t row_num) override; + void writeTotals(const Columns & columns, size_t row_num) override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + void writeBeforeExtremes() override; + void writeAfterExtremes() override; + + void writeLastSuffix() override; + + void flush() override + { + ostr->next(); + + if (validating_ostr) + out.next(); + } + + void setRowsBeforeLimit(size_t rows_before_limit_) override + { + applied_limit = true; + rows_before_limit = rows_before_limit_; + } + + void onProgress(const Progress & value) override; + + String getContentType() const override { return "application/json; charset=UTF-8"; } + +protected: + virtual void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num); + virtual void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); + + void writeRowsBeforeLimitAtLeast(); + void writeStatistics(); + + + std::unique_ptr validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character. + WriteBuffer * ostr; + + size_t field_number = 0; + size_t row_count = 0; + bool applied_limit = false; + size_t rows_before_limit = 0; + NamesAndTypes fields; + + Progress progress; + Stopwatch watch; + FormatSettings settings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp new file mode 100644 index 00000000000..8ff0d7e5724 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +class NativeInputFormatFromNativeBlockInputStream : public IInputFormat +{ +public: + NativeInputFormatFromNativeBlockInputStream(const Block & header, ReadBuffer & in) + : IInputFormat(header, in) + , stream(std::make_shared(in, header, 0)) + { + } + + String getName() const override { return "NativeInputFormatFromNativeBlockInputStream"; } + +protected: + Chunk generate() override + { + /// TODO: do something with totals and extremes. + + if (!read_prefix) + { + stream->readPrefix(); + read_prefix = true; + } + + auto block = stream->read(); + if (!block) + { + if (!read_suffix) + { + stream->readSuffix(); + read_suffix = true; + } + + return Chunk(); + } + + assertBlocksHaveEqualStructure(getPort().getHeader(), block, getName()); + block.checkNumberOfRows(); + + UInt64 num_rows = block.rows(); + return Chunk(block.getColumns(), num_rows); + } + +private: + std::shared_ptr stream; + bool read_prefix = false; + bool read_suffix = false; +}; + + +class NativeOutputFormatFromNativeBlockOutputStream : public IOutputFormat +{ +public: + NativeOutputFormatFromNativeBlockOutputStream(const Block & header, WriteBuffer & out) + : IOutputFormat(header, out) + , stream(std::make_shared(out, 0, header)) + { + } + + String getName() const override { return "NativeOutputFormatFromNativeBlockOutputStream"; } + + void setRowsBeforeLimit(size_t rows_before_limit) override + { + stream->setRowsBeforeLimit(rows_before_limit); + } + + void onProgress(const Progress & progress) override + { + stream->onProgress(progress); + } + + std::string getContentType() const override + { + return stream->getContentType(); + } + +protected: + void consume(Chunk chunk) override + { + writePrefixIfNot(); + + if (chunk) + { + + auto block = getPort(PortKind::Main).getHeader(); + block.setColumns(chunk.detachColumns()); + stream->write(block); + } + } + + void consumeTotals(Chunk chunk) override + { + writePrefixIfNot(); + + auto block = getPort(PortKind::Totals).getHeader(); + block.setColumns(chunk.detachColumns()); + stream->setTotals(block); + } + + void consumeExtremes(Chunk chunk) override + { + writePrefixIfNot(); + + auto block = getPort(PortKind::Extremes).getHeader(); + block.setColumns(chunk.detachColumns()); + stream->setExtremes(block); + } + + void finalize() override + { + writePrefixIfNot(); + writeSuffixIfNot(); + } + +private: + std::shared_ptr stream; + bool prefix_written = false; + bool suffix_written = false; + + void writePrefixIfNot() + { + if (!prefix_written) + stream->writePrefix(); + + prefix_written = true; + } + + void writeSuffixIfNot() + { + if (!suffix_written) + stream->writeSuffix(); + + suffix_written = true; + } +}; + +void registerInputFormatNativeProcessor(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("Native", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + const RowInputFormatParams &, + const FormatSettings &) + { + return std::make_shared(sample, buf); + }); +} + +void registerOutputFormatNativeProcessor(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Native", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings &) + { + return std::make_shared(sample, buf); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/NullFormat.cpp b/dbms/src/Processors/Formats/Impl/NullFormat.cpp new file mode 100644 index 00000000000..0e3af4b9f4e --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/NullFormat.cpp @@ -0,0 +1,31 @@ +#include +#include + + +namespace DB +{ + +class NullOutputFormat : public IOutputFormat +{ +public: + NullOutputFormat(const Block & header, WriteBuffer & out) : IOutputFormat(header, out) {} + + String getName() const override { return "NullOutputFormat"; } + +protected: + void consume(Chunk) override {} +}; + +void registerOutputFormatProcessorNull(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Null", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings &) + { + return std::make_shared(sample, buf); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp new file mode 100644 index 00000000000..a81838ce64a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -0,0 +1,113 @@ +#include +#include +#include +#include +#include + + +#include + + +namespace DB +{ +ODBCDriver2BlockOutputFormat::ODBCDriver2BlockOutputFormat( + WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IOutputFormat(std::move(header), out), format_settings(format_settings) +{ +} + +void writeODBCString(WriteBuffer & out, const std::string & str) +{ + writeIntBinary(Int32(str.size()), out); + out.write(str.data(), str.size()); +} + +void ODBCDriver2BlockOutputFormat::writeRow(const Block & header, const Columns & columns, size_t row_idx, std::string & buffer) +{ + size_t num_columns = columns.size(); + for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) + { + buffer.clear(); + auto & column = columns[column_idx]; + + if (column->isNullAt(row_idx)) + { + writeIntBinary(Int32(-1), out); + } + else + { + { + WriteBufferFromString text_out(buffer); + header.getByPosition(row_idx).type->serializeAsText(*column, row_idx, text_out, format_settings); + } + writeODBCString(out, buffer); + } + } +} + +void ODBCDriver2BlockOutputFormat::write(Chunk chunk, PortKind port_kind) +{ + String text_value; + auto & header = getPort(port_kind).getHeader(); + auto & columns = chunk.getColumns(); + const size_t rows = chunk.getNumRows(); + for (size_t i = 0; i < rows; ++i) + writeRow(header, columns, i, text_value); +} + +void ODBCDriver2BlockOutputFormat::consume(Chunk chunk) +{ + writePrefixIfNot(); + write(std::move(chunk), PortKind::Main); +} + +void ODBCDriver2BlockOutputFormat::consumeTotals(Chunk chunk) +{ + writePrefixIfNot(); + write(std::move(chunk), PortKind::Totals); +} + +void ODBCDriver2BlockOutputFormat::finalize() +{ + writePrefixIfNot(); +} + +void ODBCDriver2BlockOutputFormat::writePrefix() +{ + auto & header = getPort(PortKind::Main).getHeader(); + const size_t columns = header.columns(); + + /// Number of header rows. + writeIntBinary(Int32(2), out); + + /// Names of columns. + /// Number of columns + 1 for first name column. + writeIntBinary(Int32(columns + 1), out); + writeODBCString(out, "name"); + for (size_t i = 0; i < columns; ++i) + { + const ColumnWithTypeAndName & col = header.getByPosition(i); + writeODBCString(out, col.name); + } + + /// Types of columns. + writeIntBinary(Int32(columns + 1), out); + writeODBCString(out, "type"); + for (size_t i = 0; i < columns; ++i) + { + const ColumnWithTypeAndName & col = header.getByPosition(i); + writeODBCString(out, col.type->getName()); + } +} + + +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor( + "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h new file mode 100644 index 00000000000..2ff254285e6 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class WriteBuffer; + + +/** A data format designed to simplify the implementation of the ODBC driver. + * ODBC driver is designed to be build for different platforms without dependencies from the main code, + * so the format is made that way so that it can be as easy as possible to parse it. + * A header is displayed with the required information. + * The data is then output in the order of the rows. Each value is displayed as follows: length in Int32 format (-1 for NULL), then data in text form. + */ +class ODBCDriver2BlockOutputFormat final : public IOutputFormat +{ +public: + ODBCDriver2BlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "ODBCDriver2BlockOutputFormat"; } + + void consume(Chunk) override; + void consumeTotals(Chunk) override; + void finalize() override; + + std::string getContentType() const override + { + return "application/octet-stream"; + } + +private: + const FormatSettings format_settings; + bool prefix_written = false; + + void writePrefixIfNot() + { + if (!prefix_written) + writePrefix(); + + prefix_written = true; + } + + void writeRow(const Block & header, const Columns & columns, size_t row_idx, std::string & buffer); + void write(Chunk chunk, PortKind port_kind); + void writePrefix(); +}; + + + +} diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp new file mode 100644 index 00000000000..7fccc860b55 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp @@ -0,0 +1,79 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +ODBCDriverBlockOutputFormat::ODBCDriverBlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IOutputFormat(std::move(header), out), format_settings(format_settings) +{ +} + +void ODBCDriverBlockOutputFormat::consume(Chunk chunk) +{ + writePrefixIfNot(); + + const size_t num_rows = chunk.getNumRows(); + const size_t num_columns = chunk.getNumColumns(); + auto & columns = chunk.getColumns(); + auto & header = getPort(PortKind::Main).getHeader(); + String text_value; + + for (size_t i = 0; i < num_rows; ++i) + { + for (size_t j = 0; j < num_columns; ++j) + { + text_value.resize(0); + auto & column = columns[j]; + auto & type = header.getByPosition(j).type; + + { + WriteBufferFromString text_out(text_value); + type->serializeAsText(*column, i, text_out, format_settings); + } + + writeStringBinary(text_value, out); + } + } +} + +void ODBCDriverBlockOutputFormat::writePrefix() +{ + auto & header = getPort(PortKind::Main).getHeader(); + const size_t columns = header.columns(); + + /// Number of columns. + writeVarUInt(columns, out); + + /// Names and types of columns. + for (size_t i = 0; i < columns; ++i) + { + const ColumnWithTypeAndName & col = header.getByPosition(i); + + writeStringBinary(col.name, out); + writeStringBinary(col.type->getName(), out); + } +} + +void ODBCDriverBlockOutputFormat::finalize() +{ + writePrefixIfNot(); +} + +void registerOutputFormatProcessorODBCDriver(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("ODBCDriver", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.h new file mode 100644 index 00000000000..112abb81d87 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class WriteBuffer; + + +/** A data format designed to simplify the implementation of the ODBC driver. + * ODBC driver is designed to be build for different platforms without dependencies from the main code, + * so the format is made that way so that it can be as easy as possible to parse it. + * A header is displayed with the required information. + * The data is then output in the order of the rows. Each value is displayed as follows: length in VarUInt format, then data in text form. + */ +class ODBCDriverBlockOutputFormat : public IOutputFormat +{ +public: + ODBCDriverBlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "ODBCDriverBlockOutputFormat"; } + + void consume(Chunk) override; + void finalize() override; + + std::string getContentType() const override { return "application/octet-stream"; } + +private: + const FormatSettings format_settings; + bool prefix_written = false; + + void writePrefixIfNot() + { + if (!prefix_written) + writePrefix(); + + prefix_written = true; + } + + void writePrefix(); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp new file mode 100644 index 00000000000..dce500ac9b2 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -0,0 +1,281 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +PrettyBlockOutputFormat::PrettyBlockOutputFormat( + WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IOutputFormat(std::move(header), out), format_settings(format_settings) +{ + struct winsize w; + if (0 == ioctl(STDOUT_FILENO, TIOCGWINSZ, &w)) + terminal_width = w.ws_col; +} + + +/// Evaluate the visible width of the values and column names. +/// Note that number of code points is just a rough approximation of visible string width. +void PrettyBlockOutputFormat::calculateWidths( + const Block & header, const Chunk & chunk, + WidthsPerColumn & widths, Widths & max_widths, Widths & name_widths) +{ + size_t num_rows = chunk.getNumRows(); + size_t num_columns = chunk.getNumColumns(); + auto & columns = chunk.getColumns(); + + widths.resize(num_columns); + max_widths.resize_fill(num_columns); + name_widths.resize(num_columns); + + /// Calculate widths of all values. + String serialized_value; + size_t prefix = 2; // Tab character adjustment + for (size_t i = 0; i < num_columns; ++i) + { + auto & elem = header.getByPosition(i); + auto & column = columns[i]; + + widths[i].resize(num_rows); + + for (size_t j = 0; j < num_rows; ++j) + { + { + WriteBufferFromString out(serialized_value); + elem.type->serializeAsText(*column, j, out, format_settings); + } + + widths[i][j] = std::min(format_settings.pretty.max_column_pad_width, + UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix)); + max_widths[i] = std::max(max_widths[i], widths[i][j]); + } + + /// And also calculate widths for names of columns. + { + // name string doesn't contain Tab, no need to pass `prefix` + name_widths[i] = std::min(format_settings.pretty.max_column_pad_width, + UTF8::computeWidth(reinterpret_cast(elem.name.data()), elem.name.size())); + max_widths[i] = std::max(max_widths[i], name_widths[i]); + } + prefix += max_widths[i] + 3; + } +} + + +void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) +{ + UInt64 max_rows = format_settings.pretty.max_rows; + + if (total_rows >= max_rows) + { + total_rows += chunk.getNumRows(); + return; + } + + auto num_rows = chunk.getNumRows(); + auto num_columns = chunk.getNumColumns(); + auto & columns = chunk.getColumns(); + auto & header = getPort(port_kind).getHeader(); + + WidthsPerColumn widths; + Widths max_widths; + Widths name_widths; + calculateWidths(header, chunk, widths, max_widths, name_widths); + + /// Create separators + std::stringstream top_separator; + std::stringstream middle_names_separator; + std::stringstream middle_values_separator; + std::stringstream bottom_separator; + + top_separator << "┏"; + middle_names_separator << "┡"; + middle_values_separator << "├"; + bottom_separator << "└"; + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + { + top_separator << "┳"; + middle_names_separator << "╇"; + middle_values_separator << "┼"; + bottom_separator << "┴"; + } + + for (size_t j = 0; j < max_widths[i] + 2; ++j) + { + top_separator << "━"; + middle_names_separator << "━"; + middle_values_separator << "─"; + bottom_separator << "─"; + } + } + top_separator << "┓\n"; + middle_names_separator << "┩\n"; + middle_values_separator << "┤\n"; + bottom_separator << "┘\n"; + + std::string top_separator_s = top_separator.str(); + std::string middle_names_separator_s = middle_names_separator.str(); + std::string middle_values_separator_s = middle_values_separator.str(); + std::string bottom_separator_s = bottom_separator.str(); + + /// Output the block + writeString(top_separator_s, out); + + /// Names + writeCString("┃ ", out); + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + writeCString(" ┃ ", out); + + auto & col = header.getByPosition(i); + + if (format_settings.pretty.color) + writeCString("\033[1m", out); + + if (col.type->shouldAlignRightInPrettyFormats()) + { + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeChar(' ', out); + + writeString(col.name, out); + } + else + { + writeString(col.name, out); + + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeChar(' ', out); + } + + if (format_settings.pretty.color) + writeCString("\033[0m", out); + } + writeCString(" ┃\n", out); + + writeString(middle_names_separator_s, out); + + for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) + { + if (i != 0) + writeString(middle_values_separator_s, out); + + writeCString("│ ", out); + + for (size_t j = 0; j < num_columns; ++j) + { + if (j != 0) + writeCString(" │ ", out); + + auto & type = *header.getByPosition(j).type; + writeValueWithPadding(*columns[j], type, i, widths[j].empty() ? max_widths[j] : widths[j][i], max_widths[j]); + } + + writeCString(" │\n", out); + } + + writeString(bottom_separator_s, out); + + total_rows += num_rows; +} + + +void PrettyBlockOutputFormat::writeValueWithPadding( + const IColumn & column, const IDataType & type, size_t row_num, size_t value_width, size_t pad_to_width) +{ + auto writePadding = [&]() + { + for (size_t k = 0; k < pad_to_width - value_width; ++k) + writeChar(' ', out); + }; + + if (type.shouldAlignRightInPrettyFormats()) + { + writePadding(); + type.serializeAsText(column, row_num, out, format_settings); + } + else + { + type.serializeAsText(column, row_num, out, format_settings); + writePadding(); + } +} + + +void PrettyBlockOutputFormat::consume(Chunk chunk) +{ + write(chunk, PortKind::Main); +} + +void PrettyBlockOutputFormat::consumeTotals(Chunk chunk) +{ + total_rows = 0; + writeSuffixIfNot(); + writeCString("\nExtremes:\n", out); + write(chunk, PortKind::Totals); +} + +void PrettyBlockOutputFormat::consumeExtremes(Chunk chunk) +{ + total_rows = 0; + writeSuffixIfNot(); + writeCString("\nTotals:\n", out); + write(chunk, PortKind::Extremes); +} + + +void PrettyBlockOutputFormat::writeSuffix() +{ + if (total_rows >= format_settings.pretty.max_rows) + { + writeCString(" Showed first ", out); + writeIntText(format_settings.pretty.max_rows, out); + writeCString(".\n", out); + } +} + +void PrettyBlockOutputFormat::finalize() +{ + writeSuffixIfNot(); +} + + +void registerOutputFormatProcessorPretty(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Pretty", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); + + factory.registerOutputFormatProcessor("PrettyNoEscapes", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + FormatSettings changed_settings = format_settings; + changed_settings.pretty.color = false; + return std::make_shared(buf, sample, changed_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h new file mode 100644 index 00000000000..8b1ec51b660 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class WriteBuffer; +class Context; + + +/** Prints the result in the form of beautiful tables. + */ +class PrettyBlockOutputFormat : public IOutputFormat +{ +public: + /// no_escapes - do not use ANSI escape sequences - to display in the browser, not in the console. + PrettyBlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "PrettyBlockOutputFormat"; } + + void consume(Chunk) override; + void consumeTotals(Chunk) override; + void consumeExtremes(Chunk) override; + + void finalize() override; + +protected: + size_t total_rows = 0; + size_t terminal_width = 0; + bool suffix_written = false; + + const FormatSettings format_settings; + + using Widths = PODArray; + using WidthsPerColumn = std::vector; + + virtual void write(const Chunk & chunk, PortKind port_kind); + virtual void writeSuffix(); + + + void writeSuffixIfNot() + { + if (!suffix_written) + writeSuffix(); + + suffix_written = true; + } + + void calculateWidths( + const Block & header, const Chunk & chunk, + WidthsPerColumn & widths, Widths & max_widths, Widths & name_widths); + + void writeValueWithPadding( + const IColumn & column, const IDataType & type, size_t row_num, size_t value_width, size_t pad_to_width); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp new file mode 100644 index 00000000000..53fef75a97c --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -0,0 +1,167 @@ +#include +#include +///#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + +extern const int ILLEGAL_COLUMN; + +} + +void PrettyCompactBlockOutputFormat::writeHeader( + const Block & block, + const Widths & max_widths, + const Widths & name_widths) +{ + /// Names + writeCString("┌─", out); + for (size_t i = 0; i < max_widths.size(); ++i) + { + if (i != 0) + writeCString("─┬─", out); + + const ColumnWithTypeAndName & col = block.getByPosition(i); + + if (col.type->shouldAlignRightInPrettyFormats()) + { + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeCString("─", out); + + if (format_settings.pretty.color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (format_settings.pretty.color) + writeCString("\033[0m", out); + } + else + { + if (format_settings.pretty.color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (format_settings.pretty.color) + writeCString("\033[0m", out); + + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeCString("─", out); + } + } + writeCString("─┐\n", out); +} + +void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) +{ + /// Create delimiters + std::stringstream bottom_separator; + + bottom_separator << "└"; + for (size_t i = 0; i < max_widths.size(); ++i) + { + if (i != 0) + bottom_separator << "┴"; + + for (size_t j = 0; j < max_widths[i] + 2; ++j) + bottom_separator << "─"; + } + bottom_separator << "┘\n"; + + writeString(bottom_separator.str(), out); +} + +void PrettyCompactBlockOutputFormat::writeRow( + size_t row_num, + const Block & header, + const Columns & columns, + const WidthsPerColumn & widths, + const Widths & max_widths) +{ + size_t num_columns = max_widths.size(); + + writeCString("│ ", out); + + for (size_t j = 0; j < num_columns; ++j) + { + if (j != 0) + writeCString(" │ ", out); + + auto & type = *header.getByPosition(j).type; + auto & cur_widths = widths[j].empty() ? max_widths[j] : widths[j][row_num]; + writeValueWithPadding(*columns[j], type, row_num, cur_widths, max_widths[j]); + } + + writeCString(" │\n", out); +} + +void PrettyCompactBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) +{ + UInt64 max_rows = format_settings.pretty.max_rows; + + if (total_rows >= max_rows) + { + total_rows += chunk.getNumRows(); + return; + } + + size_t num_rows = chunk.getNumRows(); + auto & header = getPort(port_kind).getHeader(); + auto & columns = chunk.getColumns(); + + WidthsPerColumn widths; + Widths max_widths; + Widths name_widths; + calculateWidths(header, chunk, widths, max_widths, name_widths); + + writeHeader(header, max_widths, name_widths); + + for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) + writeRow(i, header, columns, widths, max_widths); + + writeBottom(max_widths); + + total_rows += num_rows; +} + + +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("PrettyCompact", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); + + factory.registerOutputFormatProcessor("PrettyCompactNoEscapes", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + FormatSettings changed_settings = format_settings; + changed_settings.pretty.color = false; + return std::make_shared(buf, sample, changed_settings); + }); + +/// TODO +// factory.registerOutputFormat("PrettyCompactMonoBlock", []( +// WriteBuffer & buf, +// const Block & sample, +// const Context &, +// const FormatSettings & format_settings) +// { +// BlockOutputStreamPtr impl = std::make_shared(buf, sample, format_settings); +// auto res = std::make_shared(impl, impl->getHeader(), format_settings.pretty.max_rows, 0); +// res->disableFlush(); +// return res; +// }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h new file mode 100644 index 00000000000..fb40e10b7a7 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h @@ -0,0 +1,31 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Prints the result in the form of beautiful tables, but with fewer delimiter lines. + */ +class PrettyCompactBlockOutputFormat : public PrettyBlockOutputFormat +{ +public: + PrettyCompactBlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings) + : PrettyBlockOutputFormat(out, std::move(header), format_settings) {} + + String getName() const override { return "PrettyCompactBlockOutputFormat"; } + +protected: + void write(const Chunk & chunk, PortKind port_kind) override; + void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths); + void writeBottom(const Widths & max_widths); + void writeRow( + size_t row_num, + const Block & header, + const Columns & columns, + const WidthsPerColumn & widths, + const Widths & max_widths); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp new file mode 100644 index 00000000000..2b75d867327 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) +{ + UInt64 max_rows = format_settings.pretty.max_rows; + + if (total_rows >= max_rows) + { + total_rows += chunk.getNumRows(); + return; + } + + size_t num_rows = chunk.getNumRows(); + size_t num_columns = chunk.getNumColumns(); + auto & header = getPort(port_kind).getHeader(); + auto & columns = chunk.getColumns(); + + WidthsPerColumn widths; + Widths max_widths; + Widths name_widths; + calculateWidths(header, chunk, widths, max_widths, name_widths); + + /// Do not align on too long values. + if (terminal_width > 80) + for (size_t i = 0; i < num_columns; ++i) + if (max_widths[i] > terminal_width / 2) + max_widths[i] = terminal_width / 2; + + /// Names + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + writeCString(" ", out); + + const ColumnWithTypeAndName & col = header.getByPosition(i); + + if (col.type->shouldAlignRightInPrettyFormats()) + { + for (ssize_t k = 0; k < std::max(static_cast(0), static_cast(max_widths[i] - name_widths[i])); ++k) + writeChar(' ', out); + + if (format_settings.pretty.color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (format_settings.pretty.color) + writeCString("\033[0m", out); + } + else + { + if (format_settings.pretty.color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (format_settings.pretty.color) + writeCString("\033[0m", out); + + for (ssize_t k = 0; k < std::max(static_cast(0), static_cast(max_widths[i] - name_widths[i])); ++k) + writeChar(' ', out); + } + } + writeCString("\n\n", out); + + for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) + { + for (size_t column = 0; column < num_columns; ++column) + { + if (column != 0) + writeCString(" ", out); + + auto & type = *header.getByPosition(column).type; + auto & cur_width = widths[column].empty() ? max_widths[column] : widths[column][row]; + writeValueWithPadding(*columns[column], type, row, cur_width, max_widths[column]); + } + + writeChar('\n', out); + } + + total_rows += num_rows; +} + + +void PrettySpaceBlockOutputFormat::writeSuffix() +{ + if (total_rows >= format_settings.pretty.max_rows) + { + writeCString("\nShowed first ", out); + writeIntText(format_settings.pretty.max_rows, out); + writeCString(".\n", out); + } +} + + +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("PrettySpace", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings); + }); + + factory.registerOutputFormatProcessor("PrettySpaceNoEscapes", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & format_settings) + { + FormatSettings changed_settings = format_settings; + changed_settings.pretty.color = false; + return std::make_shared(buf, sample, changed_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h new file mode 100644 index 00000000000..0002dd87d24 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h @@ -0,0 +1,24 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Prints the result, aligned with spaces. + */ +class PrettySpaceBlockOutputFormat : public PrettyBlockOutputFormat +{ +public: + PrettySpaceBlockOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings) + : PrettyBlockOutputFormat(out, std::move(header), format_settings) {} + + String getName() const override { return "PrettySpaceBlockOutputFormat"; } + +protected: + void write(const Chunk & chunk, PortKind port_kind) override; + void writeSuffix() override; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.cpp new file mode 100644 index 00000000000..2ad64a1e169 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.cpp @@ -0,0 +1,107 @@ +#include + +#include +#if USE_PROTOBUF + +#include "ProtobufBlockOutputFormat.h" + +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; +} + + +ProtobufBlockOutputFormat::ProtobufBlockOutputFormat( + WriteBuffer & out, + Block header, + const google::protobuf::Descriptor * message_type, + const FormatSettings & format_settings) + : IOutputFormat(std::move(header), out), writer(out, message_type), format_settings(format_settings) +{ +} + +void ProtobufBlockOutputFormat::consume(Chunk chunk) +{ + auto header = getPort(PortKind::Main).getHeader(); + auto & columns = chunk.getColumns(); + + std::vector header_in_write_order; + ColumnRawPtrs columns_in_write_order; + const auto & fields_in_write_order = writer.fieldsInWriteOrder(); + header_in_write_order.reserve(fields_in_write_order.size()); + columns_in_write_order.reserve(fields_in_write_order.size()); + + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + const ColumnWithTypeAndName * header_ptr = nullptr; + const IColumn * column_ptr = nullptr; + + if (header.has(field->name())) + { + auto pos = header.getPositionByName(field->name()); + + header_ptr = &header.getByPosition(pos); + column_ptr = columns[pos].get(); + } + else if (field->is_required()) + { + throw Exception( + "Output doesn't have a column named '" + field->name() + "' which is required to write the output in the protobuf format.", + ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + columns_in_write_order.emplace_back(column_ptr); + header_in_write_order.emplace_back(header_ptr); + } + + auto num_rows = chunk.getNumRows(); + for (size_t row_num = 0; row_num < num_rows; ++row_num) + { + writer.newMessage(); + auto num_columns_to_write = header_in_write_order.size(); + for (size_t ps = 0; ps < num_columns_to_write; ++ps) + { + auto * header_ptr = header_in_write_order[ps]; + auto * column_ptr = columns_in_write_order[ps]; + if (header_ptr) + { + assert(header_ptr->name == writer.currentField()->name()); + header_ptr->type->serializeProtobuf(*column_ptr, row_num, writer); + } + writer.nextField(); + } + } +} + + +void registerOutputFormatProcessorProtobuf(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor( + "Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings & format_settings) + { + const auto * message_type = ProtobufSchemas::instance().getMessageTypeForFormatSchema(FormatSchemaInfo(context, "proto")); + return std::make_shared(buf, header, message_type, format_settings); + }); +} + +} + +#else + +namespace DB +{ + class FormatFactory; + void registerOutputFormatProtobuf(FormatFactory &) {} +} + +#endif diff --git a/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.h new file mode 100644 index 00000000000..1491481e163 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufBlockOutputFormat.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Message; +} +} + + +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 + * 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' + * where schema is the name of "schema.proto" file specifying protobuf schema. + */ +class ProtobufBlockOutputFormat : public IOutputFormat +{ +public: + ProtobufBlockOutputFormat( + WriteBuffer & out, + Block header, + const google::protobuf::Descriptor * message_prototype, + const FormatSettings & format_settings); + + String getName() const override { return "ProtobufBlockOutputFormat"; } + + void consume(Chunk chunk) override; + std::string getContentType() const override { return "application/octet-stream"; } + +private: + ProtobufWriter writer; + const Block header; + const FormatSettings format_settings; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/ProtobufSchemas.cpp b/dbms/src/Processors/Formats/Impl/ProtobufSchemas.cpp new file mode 100644 index 00000000000..66a42750d89 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufSchemas.cpp @@ -0,0 +1,84 @@ +#include +#if USE_PROTOBUF + +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; +} + + +class ProtobufSchemas::ImporterWithSourceTree : public google::protobuf::compiler::MultiFileErrorCollector +{ +public: + ImporterWithSourceTree(const String & schema_directory) : importer(&disk_source_tree, this) + { + disk_source_tree.MapPath("", schema_directory); + } + + ~ImporterWithSourceTree() override = default; + + const google::protobuf::Descriptor * import(const String & schema_path, const String & message_name) + { + // Search the message type among already imported ones. + const auto * descriptor = importer.pool()->FindMessageTypeByName(message_name); + if (descriptor) + return descriptor; + + const auto * file_descriptor = importer.Import(schema_path); + // If there are parsing errors AddError() throws an exception and in this case the following line + // isn't executed. + assert(file_descriptor); + + descriptor = file_descriptor->FindMessageTypeByName(message_name); + if (!descriptor) + throw Exception( + "Not found a message named '" + message_name + "' in the schema file '" + schema_path + "'", ErrorCodes::BAD_ARGUMENTS); + + return descriptor; + } + +private: + // Overrides google::protobuf::compiler::MultiFileErrorCollector: + void AddError(const String & filename, int line, int column, const String & message) override + { + throw Exception( + "Cannot parse '" + filename + "' file, found an error at line " + std::to_string(line) + ", column " + std::to_string(column) + + ", " + message, + ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA); + } + + google::protobuf::compiler::DiskSourceTree disk_source_tree; + google::protobuf::compiler::Importer importer; +}; + + +ProtobufSchemas::ProtobufSchemas() = default; +ProtobufSchemas::~ProtobufSchemas() = default; + +const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info) +{ + auto it = importers.find(info.schemaDirectory()); + if (it == importers.end()) + it = importers.emplace(info.schemaDirectory(), std::make_unique(info.schemaDirectory())).first; + auto * importer = it->second.get(); + return importer->import(info.schemaPath(), info.messageName()); +} + +const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForColumns(const std::vector & /*columns*/) +{ + throw Exception("Using the 'Protobuf' format without schema is not implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +} + +#endif diff --git a/dbms/src/Processors/Formats/Impl/ProtobufSchemas.h b/dbms/src/Processors/Formats/Impl/ProtobufSchemas.h new file mode 100644 index 00000000000..8fc9bb5db2f --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufSchemas.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#if USE_PROTOBUF + +#include +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; +} +} + +namespace DB +{ +class FormatSchemaInfo; +struct ColumnWithTypeAndName; + +/** Keeps parsed google protobuf schemas either parsed from files or generated from DB columns. + * This class is used to handle the "Protobuf" input/output formats. + */ +class ProtobufSchemas : public ext::singleton +{ +public: + ProtobufSchemas(); + ~ProtobufSchemas(); + + /// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type. + /// The function never returns nullptr, it throws an exception if it cannot load or parse the file. + const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo & info); + + /// Generates a message type with suitable types of fields to store a block with |header|, then returns the descriptor + /// of the generated message type. + const google::protobuf::Descriptor * getMessageTypeForColumns(const std::vector & columns); + +private: + class ImporterWithSourceTree; + std::unordered_map> importers; +}; + +} + +#endif diff --git a/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.cpp b/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.cpp new file mode 100644 index 00000000000..3959f0e09b5 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.cpp @@ -0,0 +1,363 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + // Note: We cannot simply use writeVarUInt() from IO/VarInt.h here because there is one small difference: + // Google protobuf's representation of 64-bit integer contains from 1 to 10 bytes, whileas writeVarUInt() writes from 1 to 9 bytes + // because it omits the tenth byte (which is not necessary to decode actually). + void writePbVarUInt(UInt64 value, WriteBuffer & buf) + { + while (value >= 0x80) + { + buf.write(static_cast(value | 0x80)); + value >>= 7; + } + buf.write(static_cast(value)); + } + + void writePbVarInt(Int64 value, WriteBuffer & buf) + { + writePbVarUInt((static_cast(value) << 1) ^ static_cast(value >> 63), buf); + } + + void writePbVarIntNoZigZagEncoding(Int64 value, WriteBuffer & buf) { writePbVarUInt(static_cast(value), buf); } +} + + +enum ProtobufSimpleWriter::WireType : UInt32 +{ + VARINT = 0, + BITS64 = 1, + LENGTH_DELIMITED = 2, + BITS32 = 5 +}; + +ProtobufSimpleWriter::ProtobufSimpleWriter(WriteBuffer & out_) : out(out_) +{ +} + +ProtobufSimpleWriter::~ProtobufSimpleWriter() +{ + finishCurrentMessage(); +} + +void ProtobufSimpleWriter::newMessage() +{ + finishCurrentMessage(); + were_messages = true; +} + +void ProtobufSimpleWriter::finishCurrentMessage() +{ + if (!were_messages) + return; + finishCurrentField(); + current_field_number = 0; + StringRef str = message_buffer.stringRef(); + writePbVarUInt(str.size, out); + out.write(str.data, str.size); + message_buffer.restart(); +} + +void ProtobufSimpleWriter::setCurrentField(UInt32 field_number) +{ + finishCurrentField(); + assert(current_field_number < field_number); + current_field_number = field_number; + num_normal_values = 0; + num_packed_values = 0; +} + +void ProtobufSimpleWriter::finishCurrentField() +{ + if (num_packed_values) + { + assert(!num_normal_values); + StringRef str = repeated_packing_buffer.stringRef(); + if (str.size) + { + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); + repeated_packing_buffer.restart(); + } + } +} + +void ProtobufSimpleWriter::writeKey(WireType wire_type, WriteBuffer & buf) +{ + writePbVarUInt((current_field_number << 3) | wire_type, buf); +} + +void ProtobufSimpleWriter::writeInt32(Int32 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt32(UInt32 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt32(Int32 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeInt64(Int64 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt64(UInt64 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt64(Int64 value) +{ + assert(current_field_number); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed32(UInt32 value) +{ + assert(current_field_number); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed32(Int32 value) +{ + assert(current_field_number); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFloat(float value) +{ + assert(current_field_number); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed64(UInt64 value) +{ + assert(current_field_number); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed64(Int64 value) +{ + assert(current_field_number); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeDouble(double value) +{ + assert(current_field_number); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeString(const StringRef & str) +{ + assert(current_field_number); + ++num_normal_values; + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); +} + +void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value) +{ + if (value) + writeInt32(value); +} + +void ProtobufSimpleWriter::writeUInt32IfNonZero(UInt32 value) +{ + if (value) + writeUInt32(value); +} + +void ProtobufSimpleWriter::writeSInt32IfNonZero(Int32 value) +{ + if (value) + writeSInt32(value); +} + +void ProtobufSimpleWriter::writeInt64IfNonZero(Int64 value) +{ + if (value) + writeInt64(value); +} + +void ProtobufSimpleWriter::writeUInt64IfNonZero(UInt64 value) +{ + if (value) + writeUInt64(value); +} + +void ProtobufSimpleWriter::writeSInt64IfNonZero(Int64 value) +{ + if (value) + writeSInt64(value); +} + +void ProtobufSimpleWriter::writeFixed32IfNonZero(UInt32 value) +{ + if (value) + writeFixed32(value); +} + +void ProtobufSimpleWriter::writeSFixed32IfNonZero(Int32 value) +{ + if (value) + writeSFixed32(value); +} + +void ProtobufSimpleWriter::writeFloatIfNonZero(float value) +{ + if (value != 0) + writeFloat(value); +} + +void ProtobufSimpleWriter::writeFixed64IfNonZero(UInt64 value) +{ + if (value) + writeFixed64(value); +} + +void ProtobufSimpleWriter::writeSFixed64IfNonZero(Int64 value) +{ + if (value) + writeSFixed64(value); +} + +void ProtobufSimpleWriter::writeDoubleIfNonZero(double value) +{ + if (value != 0) + writeDouble(value); +} + +void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str) +{ + if (str.size) + writeString(str); +} + +void ProtobufSimpleWriter::packRepeatedInt32(Int32 value) +{ + assert(current_field_number); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value) +{ + assert(current_field_number); + writePbVarUInt(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value) +{ + assert(current_field_number); + writePbVarInt(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedInt64(Int64 value) +{ + assert(current_field_number); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value) +{ + assert(current_field_number); + writePbVarUInt(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value) +{ + assert(current_field_number); + writePbVarInt(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFloat(float value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedDouble(double value) +{ + assert(current_field_number); + writePODBinary(value, repeated_packing_buffer); + ++num_packed_values; +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.h b/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.h new file mode 100644 index 00000000000..185fc1c2289 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufSimpleWriter.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include "IO/WriteBufferFromString.h" + + +namespace DB +{ +/** Utility class to serialize protobufs. + * Knows nothing about protobuf schemas, just provides useful functions to serialize data. + * This class is written following the documentation: https://developers.google.com/protocol-buffers/docs/encoding + */ +class ProtobufSimpleWriter : private boost::noncopyable +{ +public: + ProtobufSimpleWriter(WriteBuffer & out_); + ~ProtobufSimpleWriter(); + + /// Should be called when we start writing a new message. + void newMessage(); + + /// Should be called when we start writing a new field. + /// A passed 'field_number' should be positive and greater than any previous 'field_number'. + void setCurrentField(UInt32 field_number); + UInt32 currentFieldNumber() const { return current_field_number; } + + /// Returns number of values added to the current field. + size_t numValues() const { return num_normal_values + num_packed_values; } + + void writeInt32(Int32 value); + void writeUInt32(UInt32 value); + void writeSInt32(Int32 value); + void writeInt64(Int64 value); + void writeUInt64(UInt64 value); + void writeSInt64(Int64 value); + void writeFixed32(UInt32 value); + void writeSFixed32(Int32 value); + void writeFloat(float value); + void writeFixed64(UInt64 value); + void writeSFixed64(Int64 value); + void writeDouble(double value); + void writeString(const StringRef & str); + + void writeInt32IfNonZero(Int32 value); + void writeUInt32IfNonZero(UInt32 value); + void writeSInt32IfNonZero(Int32 value); + void writeInt64IfNonZero(Int64 value); + void writeUInt64IfNonZero(UInt64 value); + void writeSInt64IfNonZero(Int64 value); + void writeFixed32IfNonZero(UInt32 value); + void writeSFixed32IfNonZero(Int32 value); + void writeFloatIfNonZero(float value); + void writeFixed64IfNonZero(UInt64 value); + void writeSFixed64IfNonZero(Int64 value); + void writeDoubleIfNonZero(double value); + void writeStringIfNotEmpty(const StringRef & str); + + void packRepeatedInt32(Int32 value); + void packRepeatedUInt32(UInt32 value); + void packRepeatedSInt32(Int32 value); + void packRepeatedInt64(Int64 value); + void packRepeatedUInt64(UInt64 value); + void packRepeatedSInt64(Int64 value); + void packRepeatedFixed32(UInt32 value); + void packRepeatedSFixed32(Int32 value); + void packRepeatedFloat(float value); + void packRepeatedFixed64(UInt64 value); + void packRepeatedSFixed64(Int64 value); + void packRepeatedDouble(double value); + +private: + void finishCurrentMessage(); + void finishCurrentField(); + + enum WireType : UInt32; + void writeKey(WireType wire_type, WriteBuffer & buf); + + WriteBuffer & out; + bool were_messages = false; + WriteBufferFromOwnString message_buffer; + UInt32 current_field_number = 0; + size_t num_normal_values = 0; + size_t num_packed_values = 0; + WriteBufferFromOwnString repeated_packing_buffer; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/ProtobufWriter.cpp b/dbms/src/Processors/Formats/Impl/ProtobufWriter.cpp new file mode 100644 index 00000000000..94c31a1c09c --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufWriter.cpp @@ -0,0 +1,801 @@ +#include "ProtobufWriter.h" +#if USE_PROTOBUF + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; + extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE; + extern const int PROTOBUF_FIELD_NOT_REPEATED; +} + + +class ProtobufWriter::Converter : private boost::noncopyable +{ +public: + Converter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : simple_writer(simple_writer_), field(field_) + { + } + + virtual ~Converter() = default; + + virtual void writeString(const StringRef &) { cannotConvertType("String"); } + + virtual void writeInt8(Int8) { cannotConvertType("Int8"); } + virtual void writeUInt8(UInt8) { cannotConvertType("UInt8"); } + virtual void writeInt16(Int16) { cannotConvertType("Int16"); } + virtual void writeUInt16(UInt16) { cannotConvertType("UInt16"); } + virtual void writeInt32(Int32) { cannotConvertType("Int32"); } + virtual void writeUInt32(UInt32) { cannotConvertType("UInt32"); } + virtual void writeInt64(Int64) { cannotConvertType("Int64"); } + virtual void writeUInt64(UInt64) { cannotConvertType("UInt64"); } + virtual void writeUInt128(const UInt128 &) { cannotConvertType("UInt128"); } + virtual void writeFloat32(Float32) { cannotConvertType("Float32"); } + virtual void writeFloat64(Float64) { cannotConvertType("Float64"); } + + virtual void prepareEnumMappingInt8(const std::vector> &) {} + virtual void prepareEnumMappingInt16(const std::vector> &) {} + virtual void writeEnumInt8(Int8) { cannotConvertType("Enum"); } + virtual void writeEnumInt16(Int16) { cannotConvertType("Enum"); } + + virtual void writeUUID(const UUID &) { cannotConvertType("UUID"); } + virtual void writeDate(DayNum) { cannotConvertType("Date"); } + virtual void writeDateTime(time_t) { cannotConvertType("DateTime"); } + + virtual void writeDecimal32(Decimal32, UInt32) { cannotConvertType("Decimal32"); } + virtual void writeDecimal64(Decimal64, UInt32) { cannotConvertType("Decimal64"); } + virtual void writeDecimal128(const Decimal128 &, UInt32) { cannotConvertType("Decimal128"); } + + virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) { cannotConvertType("AggregateFunction"); } + +protected: + void cannotConvertType(const String & type_name) + { + throw Exception( + "Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + void cannotConvertValue(const String & value) + { + throw Exception( + "Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + template + To numericCast(From value) + { + if constexpr (std::is_same_v) + return value; + To result; + try + { + result = boost::numeric_cast(value); + } + catch (boost::numeric::bad_numeric_cast &) + { + cannotConvertValue(toString(value)); + } + return result; + } + + template + To parseFromString(const StringRef & str) + { + To result; + try + { + result = ::DB::parse(str.data, str.size); + } + catch (...) + { + cannotConvertValue(str.toString()); + } + return result; + } + + bool packRepeated() const + { + if (!field->is_repeated()) + return false; + if (field->options().has_packed()) + return field->options().packed(); + return field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3; + } + + bool skipNullValue() const + { + return field->is_optional() && (field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3); + } + + ProtobufSimpleWriter & simple_writer; + const google::protobuf::FieldDescriptor * field; +}; + + +class ProtobufWriter::ToStringConverter : public Converter +{ +public: + ToStringConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(str); } + + void writeInt8(Int8 value) override { convertToStringAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToStringAndWriteField(value); } + void writeInt16(Int16 value) override { convertToStringAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToStringAndWriteField(value); } + void writeInt32(Int32 value) override { convertToStringAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToStringAndWriteField(value); } + void writeInt64(Int64 value) override { convertToStringAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToStringAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToStringAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToStringAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_name_map->find(value); + if (it == enum_value_to_name_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + + void writeUUID(const UUID & uuid) override { convertToStringAndWriteField(uuid); } + void writeDate(DayNum date) override { convertToStringAndWriteField(date); } + + void writeDateTime(time_t tm) override + { + writeDateTimeText(tm, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal128(const Decimal128 & decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) override + { + function->serialize(place, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + +private: + template + void convertToStringAndWriteField(T value) + { + writeText(value, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + writeText(decimal, scale, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void prepareEnumValueToNameMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_name_map.has_value()) + return; + enum_value_to_name_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + enum_value_to_name_map->emplace(name_value_pair.second, name_value_pair.first); + } + + void writeField(const StringRef & str) { (simple_writer.*write_field_function)(str); } + + void initWriteFieldFunction() + { + write_field_function = skipNullValue() ? &ProtobufSimpleWriter::writeStringIfNotEmpty : &ProtobufSimpleWriter::writeString; + } + + void (ProtobufSimpleWriter::*write_field_function)(const StringRef & str); + WriteBufferFromOwnString text_buffer; + std::optional> enum_value_to_name_map; +}; + + +template +class ProtobufWriter::ToNumberConverter : public Converter +{ +public: + ToNumberConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(parseFromString(str)); } + + void writeInt8(Int8 value) override { castNumericAndWriteField(value); } + void writeUInt8(UInt8 value) override { castNumericAndWriteField(value); } + void writeInt16(Int16 value) override { castNumericAndWriteField(value); } + void writeUInt16(UInt16 value) override { castNumericAndWriteField(value); } + void writeInt32(Int32 value) override { castNumericAndWriteField(value); } + void writeUInt32(UInt32 value) override { castNumericAndWriteField(value); } + void writeInt64(Int64 value) override { castNumericAndWriteField(value); } + void writeUInt64(UInt64 value) override { castNumericAndWriteField(value); } + void writeFloat32(Float32 value) override { castNumericAndWriteField(value); } + void writeFloat64(Float64 value) override { castNumericAndWriteField(value); } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + if constexpr (!std::is_integral_v) + cannotConvertType("Enum"); // It's not correct to convert enum to floating point. + castNumericAndWriteField(value); + } + + void writeDate(DayNum date) override { castNumericAndWriteField(static_cast(date)); } + void writeDateTime(time_t tm) override { castNumericAndWriteField(tm); } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + +private: + template + void castNumericAndWriteField(From value) + { + writeField(numericCast(value)); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + if constexpr (std::is_integral_v) + castNumericAndWriteField(decimal.value / decimalScaleMultiplier(scale)); + else + castNumericAndWriteField(double(decimal.value) * pow(10., -double(scale))); + } + + void writeField(T value) { (simple_writer.*write_field_function)(value); } + + void initWriteFieldFunction() + { + if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt32IfNonZero : &ProtobufSimpleWriter::writeInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt32IfNonZero : &ProtobufSimpleWriter::writeSInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed32IfNonZero : &ProtobufSimpleWriter::writeSFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed32IfNonZero : &ProtobufSimpleWriter::writeFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt64IfNonZero : &ProtobufSimpleWriter::writeInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt64IfNonZero : &ProtobufSimpleWriter::writeSInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed64IfNonZero : &ProtobufSimpleWriter::writeSFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt64IfNonZero : &ProtobufSimpleWriter::writeUInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed64IfNonZero : &ProtobufSimpleWriter::writeFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFloat + : (skipNullValue() ? &ProtobufSimpleWriter::writeFloatIfNonZero : &ProtobufSimpleWriter::writeFloat); + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedDouble + : (skipNullValue() ? &ProtobufSimpleWriter::writeDoubleIfNonZero : &ProtobufSimpleWriter::writeDouble); + } + else + { + assert(false); + } + } + + void (ProtobufSimpleWriter::*write_field_function)(T value); +}; + + +class ProtobufWriter::ToBoolConverter : public Converter +{ +public: + ToBoolConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + if (str == "true") + writeField(true); + else if (str == "false") + writeField(false); + else + cannotConvertValue(str.toString()); + } + + void writeInt8(Int8 value) override { convertToBoolAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToBoolAndWriteField(value); } + void writeInt16(Int16 value) override { convertToBoolAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToBoolAndWriteField(value); } + void writeInt32(Int32 value) override { convertToBoolAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToBoolAndWriteField(value); } + void writeInt64(Int64 value) override { convertToBoolAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToBoolAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToBoolAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToBoolAndWriteField(value); } + void writeDecimal32(Decimal32 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal64(Decimal64 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal128(const Decimal128 & decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + +private: + template + void convertToBoolAndWriteField(T value) + { + writeField(static_cast(value)); + } + + void writeField(bool b) { (simple_writer.*write_field_function)(b); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 b); +}; + + +class ProtobufWriter::ToEnumConverter : public Converter +{ +public: + ToEnumConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + prepareEnumNameToPbNumberMap(); + auto it = enum_name_to_pbnumber_map->find(str); + if (it == enum_name_to_pbnumber_map->end()) + cannotConvertValue(str.toString()); + writeField(it->second); + } + + void writeInt8(Int8 value) override { convertToEnumAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToEnumAndWriteField(value); } + void writeInt16(Int16 value) override { convertToEnumAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToEnumAndWriteField(value); } + void writeInt32(Int32 value) override { convertToEnumAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToEnumAndWriteField(value); } + void writeInt64(Int64 value) override { convertToEnumAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToEnumAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_pbnumber_map->find(value); + if (it == enum_value_to_pbnumber_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + +private: + template + void convertToEnumAndWriteField(T value) + { + const auto * enum_descriptor = field->enum_type()->FindValueByNumber(numericCast(value)); + if (!enum_descriptor) + cannotConvertValue(toString(value)); + writeField(enum_descriptor->number()); + } + + void prepareEnumNameToPbNumberMap() + { + if (enum_name_to_pbnumber_map.has_value()) + return; + enum_name_to_pbnumber_map.emplace(); + const auto * enum_type = field->enum_type(); + for (int i = 0; i != enum_type->value_count(); ++i) + { + const auto * enum_value = enum_type->value(i); + enum_name_to_pbnumber_map->emplace(enum_value->name(), enum_value->number()); + } + } + + template + void prepareEnumValueToPbNumberMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_pbnumber_map.has_value()) + return; + enum_value_to_pbnumber_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + { + Int16 value = name_value_pair.second; + const auto * enum_descriptor = field->enum_type()->FindValueByName(name_value_pair.first); + if (enum_descriptor) + enum_value_to_pbnumber_map->emplace(value, enum_descriptor->number()); + } + } + + void writeField(int enum_number) { (simple_writer.*write_field_function)(enum_number); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 enum_number); + std::optional> enum_name_to_pbnumber_map; + std::optional> enum_value_to_pbnumber_map; +}; + + +ProtobufWriter::ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) : simple_writer(out) +{ + enumerateFieldsInWriteOrder(message_type); + createConverters(); +} + +ProtobufWriter::~ProtobufWriter() +{ + finishCurrentMessage(); +} + +void ProtobufWriter::enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type) +{ + assert(fields_in_write_order.empty()); + fields_in_write_order.reserve(message_type->field_count()); + for (int i = 0; i < message_type->field_count(); ++i) + fields_in_write_order.emplace_back(message_type->field(i)); + + std::sort( + fields_in_write_order.begin(), + fields_in_write_order.end(), + [](const google::protobuf::FieldDescriptor * left, const google::protobuf::FieldDescriptor * right) + { + return left->number() < right->number(); + }); +} + +void ProtobufWriter::createConverters() +{ + assert(converters.empty()); + converters.reserve(fields_in_write_order.size()); + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + std::unique_ptr converter; + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + case google::protobuf::FieldDescriptor::TYPE_SINT32: + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT32: + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_INT64: + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + case google::protobuf::FieldDescriptor::TYPE_SINT64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT64: + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_FLOAT: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_DOUBLE: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_BOOL: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_ENUM: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_STRING: + case google::protobuf::FieldDescriptor::TYPE_BYTES: + converter = std::make_unique(simple_writer, field); + break; + default: + throw Exception(String("Protobuf type '") + field->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED); + } + converters.emplace_back(std::move(converter)); + } +} + +const std::vector & ProtobufWriter::fieldsInWriteOrder() const +{ + return fields_in_write_order; +} + +void ProtobufWriter::newMessage() +{ + finishCurrentMessage(); + simple_writer.newMessage(); + if (fields_in_write_order.empty()) + return; + + current_field_index = 0; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); +} + +void ProtobufWriter::finishCurrentMessage() +{ + if (current_field) + { + assert(current_field_index == fields_in_write_order.size() - 1); + finishCurrentField(); + } +} + +bool ProtobufWriter::nextField() +{ + if (current_field_index == fields_in_write_order.size() - 1) + return false; + + finishCurrentField(); + + ++current_field_index; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); + return true; +} + +void ProtobufWriter::finishCurrentField() +{ + assert(current_field); + size_t num_values = simple_writer.numValues(); + if (num_values == 0) + { + if (current_field->is_required()) + throw Exception( + "No data for the required field '" + current_field->name() + "'", ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + else if (num_values > 1 && !current_field->is_repeated()) + { + throw Exception( + "Cannot write more than single value to the non-repeated field '" + current_field->name() + "'", + ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); + } +} + +void ProtobufWriter::writeNumber(Int8 value) +{ + current_converter->writeInt8(value); +} + +void ProtobufWriter::writeNumber(UInt8 value) +{ + current_converter->writeUInt8(value); +} + +void ProtobufWriter::writeNumber(Int16 value) +{ + current_converter->writeInt16(value); +} + +void ProtobufWriter::writeNumber(UInt16 value) +{ + current_converter->writeUInt16(value); +} + +void ProtobufWriter::writeNumber(Int32 value) +{ + current_converter->writeInt32(value); +} + +void ProtobufWriter::writeNumber(UInt32 value) +{ + current_converter->writeUInt32(value); +} + +void ProtobufWriter::writeNumber(Int64 value) +{ + current_converter->writeInt64(value); +} + +void ProtobufWriter::writeNumber(UInt64 value) +{ + current_converter->writeUInt64(value); +} + +void ProtobufWriter::writeNumber(UInt128 value) +{ + current_converter->writeUInt128(value); +} + +void ProtobufWriter::writeNumber(Float32 value) +{ + current_converter->writeFloat32(value); +} + +void ProtobufWriter::writeNumber(Float64 value) +{ + current_converter->writeFloat64(value); +} + +void ProtobufWriter::writeString(const StringRef & str) +{ + current_converter->writeString(str); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt8(enum_values); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt16(enum_values); +} + +void ProtobufWriter::writeEnum(Int8 value) +{ + current_converter->writeEnumInt8(value); +} + +void ProtobufWriter::writeEnum(Int16 value) +{ + current_converter->writeEnumInt16(value); +} + +void ProtobufWriter::writeUUID(const UUID & uuid) +{ + current_converter->writeUUID(uuid); +} + +void ProtobufWriter::writeDate(DayNum date) +{ + current_converter->writeDate(date); +} + +void ProtobufWriter::writeDateTime(time_t tm) +{ + current_converter->writeDateTime(tm); +} + +void ProtobufWriter::writeDecimal(Decimal32 decimal, UInt32 scale) +{ + current_converter->writeDecimal32(decimal, scale); +} + +void ProtobufWriter::writeDecimal(Decimal64 decimal, UInt32 scale) +{ + current_converter->writeDecimal64(decimal, scale); +} + +void ProtobufWriter::writeDecimal(const Decimal128 & decimal, UInt32 scale) +{ + current_converter->writeDecimal128(decimal, scale); +} + +void ProtobufWriter::writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) +{ + current_converter->writeAggregateFunction(function, place); +} + +} + +#endif diff --git a/dbms/src/Processors/Formats/Impl/ProtobufWriter.h b/dbms/src/Processors/Formats/Impl/ProtobufWriter.h new file mode 100644 index 00000000000..d4b17cd692a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ProtobufWriter.h @@ -0,0 +1,126 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; + class FieldDescriptor; +} +} + +#if USE_PROTOBUF +# define EMPTY_DEF +# define EMPTY_DEF_RET(a) +#else +# define EMPTY_DEF {} +# define EMPTY_DEF_RET(a) {return a;} +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# pragma GCC diagnostic ignored "-Wextra-semi" +#endif + + +namespace DB +{ +class IAggregateFunction; +using AggregateFunctionPtr = std::shared_ptr; +using ConstAggregateDataPtr = const char *; + + +/** Serializes a protobuf, tries to cast types if necessarily. + */ +class ProtobufWriter : private boost::noncopyable +{ +public: + ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) EMPTY_DEF; + ~ProtobufWriter() EMPTY_DEF; + + /// Returns fields of the protobuf schema sorted by their numbers. + const std::vector & fieldsInWriteOrder() const; + + /// Should be called when we start writing a new message. + void newMessage() EMPTY_DEF; + + /// Should be called when we start writing a new field. + /// Returns false if there is no more fields in the message type. + bool nextField() EMPTY_DEF_RET(false); + + /// Returns the current field of the message type. + /// The value returned by this function changes after calling nextField() or newMessage(). +#if USE_PROTOBUF + const google::protobuf::FieldDescriptor * currentField() const { return current_field; } +#endif + + void writeNumber(Int8 value) EMPTY_DEF; + void writeNumber(UInt8 value) EMPTY_DEF; + void writeNumber(Int16 value) EMPTY_DEF; + void writeNumber(UInt16 value) EMPTY_DEF; + void writeNumber(Int32 value) EMPTY_DEF; + void writeNumber(UInt32 value) EMPTY_DEF; + void writeNumber(Int64 value) EMPTY_DEF; + void writeNumber(UInt64 value) EMPTY_DEF; + void writeNumber(UInt128 value) EMPTY_DEF; + void writeNumber(Float32 value) EMPTY_DEF; + void writeNumber(Float64 value) EMPTY_DEF; + + void writeString(const StringRef & value) EMPTY_DEF; + + void prepareEnumMapping(const std::vector> & name_value_pairs) EMPTY_DEF; + void prepareEnumMapping(const std::vector> & name_value_pairs) EMPTY_DEF; + void writeEnum(Int8 value) EMPTY_DEF; + void writeEnum(Int16 value) EMPTY_DEF; + + void writeUUID(const UUID & value) EMPTY_DEF; + void writeDate(DayNum date) EMPTY_DEF; + void writeDateTime(time_t tm) EMPTY_DEF; + + void writeDecimal(Decimal32 decimal, UInt32 scale) EMPTY_DEF; + void writeDecimal(Decimal64 decimal, UInt32 scale) EMPTY_DEF; + void writeDecimal(const Decimal128 & decimal, UInt32 scale) EMPTY_DEF; + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) EMPTY_DEF; + +private: +#if USE_PROTOBUF + + void enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type); + void createConverters(); + + void finishCurrentMessage(); + void finishCurrentField(); + + ProtobufSimpleWriter simple_writer; + std::vector fields_in_write_order; + size_t current_field_index = -1; + const google::protobuf::FieldDescriptor * current_field = nullptr; + + class Converter; + class ToStringConverter; + template + class ToNumberConverter; + class ToBoolConverter; + class ToEnumConverter; + + std::vector> converters; + Converter * current_converter = nullptr; + +#endif +}; + +} + +#if !USE_PROTOBUF +# undef EMPTY_DEF +# undef EMPTY_DEF_RET +# pragma GCC diagnostic pop +#endif diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp new file mode 100644 index 00000000000..e8711f3819c --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -0,0 +1,207 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_READ_ALL_DATA; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; +} + + +TSKVRowInputFormat::TSKVRowInputFormat(ReadBuffer & in, Block header, Params params, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header), in, params), format_settings(format_settings), name_map(header.columns()) +{ + /// In this format, we assume that column name cannot contain BOM, + /// so BOM at beginning of stream cannot be confused with name of field, and it is safe to skip it. + skipBOMIfExists(in); + + size_t num_columns = header.columns(); + for (size_t i = 0; i < num_columns; ++i) + name_map[header.safeGetByPosition(i).name] = i; /// NOTE You could place names more cache-locally. +} + + +/** Read the field name in the `tskv` format. + * Return true if the field is followed by an equal sign, + * otherwise (field with no value) return false. + * The reference to the field name will be written to `ref`. + * A temporary `tmp` buffer can also be used to copy the field name to it. + * When reading, skips the name and the equal sign after it. + */ +static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) +{ + tmp.clear(); + + while (!buf.eof()) + { + const char * next_pos = find_first_symbols<'\t', '\n', '\\', '='>(buf.position(), buf.buffer().end()); + + if (next_pos == buf.buffer().end()) + { + tmp.append(buf.position(), next_pos - buf.position()); + buf.next(); + continue; + } + + /// Came to the end of the name. + if (*next_pos != '\\') + { + bool have_value = *next_pos == '='; + if (tmp.empty()) + { + /// No need to copy data, you can refer directly to the `buf`. + ref = StringRef(buf.position(), next_pos - buf.position()); + buf.position() += next_pos + have_value - buf.position(); + } + else + { + /// Copy the data to a temporary string and return a reference to it. + tmp.append(buf.position(), next_pos - buf.position()); + buf.position() += next_pos + have_value - buf.position(); + ref = StringRef(tmp); + } + return have_value; + } + /// The name has an escape sequence. + else + { + tmp.append(buf.position(), next_pos - buf.position()); + buf.position() += next_pos + 1 - buf.position(); + if (buf.eof()) + throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE); + + tmp.push_back(parseEscapeSequence(*buf.position())); + ++buf.position(); + continue; + } + } + + throw Exception("Unexpected end of stream while reading key name from TSKV format", ErrorCodes::CANNOT_READ_ALL_DATA); +} + + +bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + if (in.eof()) + return false; + + auto & header = getPort().getHeader(); + size_t num_columns = columns.size(); + + /// Set of columns for which the values were read. The rest will be filled with default values. + read_columns.assign(num_columns, false); + + if (unlikely(*in.position() == '\n')) + { + /// An empty string. It is permissible, but it is unclear why. + ++in.position(); + } + else + { + while (true) + { + StringRef name_ref; + bool has_value = readName(in, name_ref, name_buf); + ssize_t index = -1; + + if (has_value) + { + /// NOTE Optimization is possible by caching the order of fields (which is almost always the same) + /// and quickly checking for the next expected field, instead of searching the hash table. + + auto it = name_map.find(name_ref); + if (name_map.end() == it) + { + if (!format_settings.skip_unknown_fields) + throw Exception("Unknown field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); + + /// If the key is not found, skip the value. + NullSink sink; + readEscapedStringInto(sink, in); + } + else + { + index = it->second; + + if (read_columns[index]) + throw Exception("Duplicate field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); + + read_columns[index] = true; + + header.getByPosition(index).type->deserializeAsTextEscaped(*columns[index], in, format_settings); + } + } + else + { + /// The only thing that can go without value is `tskv` fragment that is ignored. + if (!(name_ref.size == 4 && 0 == memcmp(name_ref.data, "tskv", 4))) + throw Exception("Found field without value while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); + } + + if (in.eof()) + { + throw Exception("Unexpected end of stream after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_READ_ALL_DATA); + } + else if (*in.position() == '\t') + { + ++in.position(); + continue; + } + else if (*in.position() == '\n') + { + ++in.position(); + break; + } + else + { + /// Possibly a garbage was written into column, remove it + if (index >= 0) + { + columns[index]->popBack(1); + read_columns[index] = false; + } + + throw Exception("Found garbage after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED); + } + } + } + + /// Fill in the not met columns with default values. + for (size_t i = 0; i < num_columns; ++i) + if (!read_columns[i]) + header.getByPosition(i).type->insertDefaultInto(*columns[i]); + + /// return info about defaults set + ext.read_columns = read_columns; + + return true; +} + + +void TSKVRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + + +void registerInputFormatProcessorTSKV(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("TSKV", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h new file mode 100644 index 00000000000..d2dd150ca7b --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class ReadBuffer; + + +/** Stream for reading data in TSKV format. + * TSKV is a very inefficient data format. + * Similar to TSV, but each field is written as key=value. + * Fields can be listed in any order (including, in different lines there may be different order), + * and some fields may be missing. + * An equal sign can be escaped in the field name. + * Also, as an additional element there may be a useless tskv fragment - it needs to be ignored. + */ +class TSKVRowInputFormat : public IRowInputFormat +{ +public: + TSKVRowInputFormat(ReadBuffer & in, Block header, Params params, const FormatSettings & format_settings); + + String getName() const override { return "TSKVRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + +private: + const FormatSettings format_settings; + + /// Buffer for the read from the stream the field name. Used when you have to copy it. + String name_buf; + + /// Hash table matching `field name -> position in the block`. NOTE You can use perfect hash map. + using NameMap = HashMap; + NameMap name_map; + + std::vector read_columns; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp new file mode 100644 index 00000000000..4b1abb7ede1 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -0,0 +1,55 @@ +#include +#include +#include +#include + + + +namespace DB +{ + +TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings_) + : TabSeparatedRowOutputFormat(out, std::move(header), false, false, format_settings_) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); + + for (auto & field : fields) + { + WriteBufferFromOwnString wb; + writeAnyEscapedString<'='>(field.name.data(), field.name.data() + field.name.size(), wb); + writeCString("=", wb); + field.name = wb.str(); + } +} + + +void TSKVRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + writeString(fields[field_number].name, out); + type.serializeAsTextEscaped(column, row_num, out, format_settings); + ++field_number; +} + + +void TSKVRowOutputFormat::writeRowEndDelimiter() +{ + writeChar('\n', out); + field_number = 0; +} + + +void registerOutputFormatProcessorTSKV(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("TSKV", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h new file mode 100644 index 00000000000..9e48c126b3f --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in the TSKV format. + * TSKV is similar to TabSeparated, but before every value, its name and equal sign are specified: name=value. + * This format is very inefficient. + */ +class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat +{ +public: + TSKVRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "TSKVRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeRowEndDelimiter() override; + +protected: + NamesAndTypes fields; + size_t field_number = 0; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h new file mode 100644 index 00000000000..3937422b78f --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** A stream for outputting data in tsv format, but without escaping individual values. + * (That is, the output is irreversible.) + */ +class TabSeparatedRawRowOutputFormat : public TabSeparatedRowOutputFormat +{ +public: + TabSeparatedRawRowOutputFormat(WriteBuffer & out, Block header, bool with_names, bool with_types, const FormatSettings & format_settings) + : TabSeparatedRowOutputFormat(out, std::move(header), with_names, with_types, format_settings) {} + + String getName() const override { return "TabSeparatedRawRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override + { + type.serializeAsText(column, row_num, out, format_settings); + } +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp new file mode 100644 index 00000000000..7881490a321 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -0,0 +1,370 @@ +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; +} + + +TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( + ReadBuffer & in, Block header, bool with_names, bool with_types, Params params, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header), in, params), with_names(with_names), with_types(with_types), format_settings(format_settings) +{ + auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + data_types.resize(num_columns); + for (size_t i = 0; i < num_columns; ++i) + data_types[i] = sample.safeGetByPosition(i).type; +} + + +void TabSeparatedRowInputFormat::readPrefix() +{ + auto & header = getPort().getHeader(); + size_t num_columns = header.columns(); + String tmp; + + if (with_names || with_types) + { + /// In this format, we assume that column name or type cannot contain BOM, + /// so, if format has header, + /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. + skipBOMIfExists(in); + } + + if (with_names) + { + for (size_t i = 0; i < num_columns; ++i) + { + readEscapedString(tmp, in); + assertChar(i == num_columns - 1 ? '\n' : '\t', in); + } + } + + if (with_types) + { + for (size_t i = 0; i < num_columns; ++i) + { + readEscapedString(tmp, in); + assertChar(i == num_columns - 1 ? '\n' : '\t', in); + } + } +} + + +/** Check for a common error case - usage of Windows line feed. + */ +static void checkForCarriageReturn(ReadBuffer & in) +{ + if (in.position()[0] == '\r' || (in.position() != in.buffer().begin() && in.position()[-1] == '\r')) + throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row." + "\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format." + " You must transform your file to Unix format." + "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.", + ErrorCodes::INCORRECT_DATA); +} + + +bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + if (in.eof()) + return false; + + updateDiagnosticInfo(); + + size_t size = data_types.size(); + + for (size_t i = 0; i < size; ++i) + { + data_types[i]->deserializeAsTextEscaped(*columns[i], in, format_settings); + + /// skip separators + if (i + 1 == size) + { + if (!in.eof()) + { + if (unlikely(row_num == 1)) + checkForCarriageReturn(in); + + assertChar('\n', in); + } + } + else + assertChar('\t', in); + } + + return true; +} + + +String TabSeparatedRowInputFormat::getDiagnosticInfo() +{ + if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed. + return {}; + + auto & header = getPort().getHeader(); + WriteBufferFromOwnString out; + MutableColumns columns = header.cloneEmptyColumns(); + + /// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer. + size_t bytes_read_at_start_of_buffer = in.count() - in.offset(); + if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) + { + out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; + return out.str(); + } + + size_t max_length_of_column_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) + max_length_of_column_name = header.safeGetByPosition(i).name.size(); + + size_t max_length_of_data_type_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) + max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); + + /// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information. + + if (pos_of_prev_row) + { + in.position() = pos_of_prev_row; + + out << "\nRow " << (row_num - 1) << ":\n"; + if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) + return out.str(); + } + else + { + if (!pos_of_current_row) + { + out << "Could not print diagnostic info because parsing of data hasn't started.\n"; + return out.str(); + } + + in.position() = pos_of_current_row; + } + + out << "\nRow " << row_num << ":\n"; + parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); + out << "\n"; + + return out.str(); +} + + +bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, + WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) +{ + auto & header = getPort().getHeader(); + size_t size = data_types.size(); + for (size_t i = 0; i < size; ++i) + { + if (i == 0 && in.eof()) + { + out << "\n"; + return false; + } + + out << "Column " << i << ", " << std::string((i < 10 ? 2 : i < 100 ? 1 : 0), ' ') + << "name: " << header.safeGetByPosition(i).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(i).name.size(), ' ') + << "type: " << data_types[i]->getName() << ", " << std::string(max_length_of_data_type_name - data_types[i]->getName().size(), ' '); + + auto prev_position = in.position(); + std::exception_ptr exception; + + try + { + data_types[i]->deserializeAsTextEscaped(*columns[i], in, format_settings); + } + catch (...) + { + exception = std::current_exception(); + } + + auto curr_position = in.position(); + + if (curr_position < prev_position) + throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); + + if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i])) + { + /// An empty string instead of a value. + if (curr_position == prev_position) + { + out << "ERROR: text "; + verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out); + out << " is not like " << data_types[i]->getName() << "\n"; + return false; + } + } + + out << "parsed text: "; + verbosePrintString(prev_position, curr_position, out); + + if (exception) + { + if (data_types[i]->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (data_types[i]->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + else + out << "ERROR\n"; + return false; + } + + out << "\n"; + + if (data_types[i]->haveMaximumSizeOfValue()) + { + if (*curr_position != '\n' && *curr_position != '\t') + { + out << "ERROR: garbage after " << data_types[i]->getName() << ": "; + verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out); + out << "\n"; + + if (data_types[i]->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (data_types[i]->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + + return false; + } + } + + /// Delimiters + if (i + 1 == size) + { + if (!in.eof()) + { + try + { + assertChar('\n', in); + } + catch (const DB::Exception &) + { + if (*in.position() == '\t') + { + out << "ERROR: Tab found where line feed is expected." + " It's like your file has more columns than expected.\n" + "And if your file have right number of columns, maybe it have unescaped tab in value.\n"; + } + else if (*in.position() == '\r') + { + out << "ERROR: Carriage return found where line feed is expected." + " It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n"; + } + else + { + out << "ERROR: There is no line feed. "; + verbosePrintString(in.position(), in.position() + 1, out); + out << " found instead.\n"; + } + return false; + } + } + } + else + { + try + { + assertChar('\t', in); + } + catch (const DB::Exception &) + { + if (*in.position() == '\n') + { + out << "ERROR: Line feed found where tab is expected." + " It's like your file has less columns than expected.\n" + "And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n"; + } + else if (*in.position() == '\r') + { + out << "ERROR: Carriage return found where tab is expected.\n"; + } + else + { + out << "ERROR: There is no tab. "; + verbosePrintString(in.position(), in.position() + 1, out); + out << " found instead.\n"; + } + return false; + } + } + } + + return true; +} + + +void TabSeparatedRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + + +void TabSeparatedRowInputFormat::updateDiagnosticInfo() +{ + ++row_num; + + bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; + bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset(); + + pos_of_prev_row = pos_of_current_row; + pos_of_current_row = in.position(); +} + + +void registerInputFormatProcessorTabSeparated(FormatFactory & factory) +{ + for (auto name : {"TabSeparated", "TSV"}) + { + factory.registerInputFormatProcessor(name, []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, false, false, params, settings); + }); + } + + for (auto name : {"TabSeparatedWithNames", "TSVWithNames"}) + { + factory.registerInputFormatProcessor(name, []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, true, false, params, settings); + }); + } + + for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) + { + factory.registerInputFormatProcessor(name, []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, true, true, params, settings); + }); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h new file mode 100644 index 00000000000..363a2fb30b4 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class ReadBuffer; + + +/** A stream to input data in tsv format. + */ +class TabSeparatedRowInputFormat : public IRowInputFormat +{ +public: + /** with_names - the first line is the header with the names of the columns + * with_types - on the next line header with type names + */ + TabSeparatedRowInputFormat( + ReadBuffer & in, Block header, bool with_names, bool with_types, Params params, const FormatSettings & format_settings); + + String getName() const override { return "TabSeparatedRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + void readPrefix() override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + + std::string getDiagnosticInfo() override; + +private: + bool with_names; + bool with_types; + const FormatSettings format_settings; + DataTypes data_types; + + /// For convenient diagnostics in case of an error. + + size_t row_num = 0; + + /// How many bytes were read, not counting those still in the buffer. + size_t bytes_read_at_start_of_buffer_on_current_row = 0; + size_t bytes_read_at_start_of_buffer_on_prev_row = 0; + + char * pos_of_current_row = nullptr; + char * pos_of_prev_row = nullptr; + + void updateDiagnosticInfo(); + + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, + WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp new file mode 100644 index 00000000000..1b940d7a2e0 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include + + +namespace DB +{ + +TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( + WriteBuffer & out, Block header, bool with_names, bool with_types, const FormatSettings & format_settings) + : IRowOutputFormat(std::move(header), out), with_names(with_names), with_types(with_types), format_settings(format_settings) +{ +} + + +void TabSeparatedRowOutputFormat::writePrefix() +{ + auto & header = getPort(PortKind::Main).getHeader(); + size_t columns = header.columns(); + + if (with_names) + { + for (size_t i = 0; i < columns; ++i) + { + writeEscapedString(header.safeGetByPosition(i).name, out); + writeChar(i == columns - 1 ? '\n' : '\t', out); + } + } + + if (with_types) + { + for (size_t i = 0; i < columns; ++i) + { + writeEscapedString(header.safeGetByPosition(i).type->getName(), out); + writeChar(i == columns - 1 ? '\n' : '\t', out); + } + } +} + + +void TabSeparatedRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextEscaped(column, row_num, out, format_settings); +} + + +void TabSeparatedRowOutputFormat::writeFieldDelimiter() +{ + writeChar('\t', out); +} + + +void TabSeparatedRowOutputFormat::writeRowEndDelimiter() +{ + writeChar('\n', out); +} + +void TabSeparatedRowOutputFormat::writeBeforeTotals() +{ + writeChar('\n', out); +} + +void TabSeparatedRowOutputFormat::writeBeforeExtremes() +{ + writeChar('\n', out); +} + + +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) +{ + for (auto name : {"TabSeparated", "TSV"}) + { + factory.registerOutputFormatProcessor(name, []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, false, false, settings); + }); + } + + for (auto name : {"TabSeparatedRaw", "TSVRaw"}) + { + factory.registerOutputFormatProcessor(name, []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, false, false, settings); + }); + } + + for (auto name : {"TabSeparatedWithNames", "TSVWithNames"}) + { + factory.registerOutputFormatProcessor(name, []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, true, false, settings); + }); + } + + for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) + { + factory.registerOutputFormatProcessor(name, []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, true, true, settings); + }); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h new file mode 100644 index 00000000000..4e8f40c6c29 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class WriteBuffer; + +/** A stream for outputting data in tsv format. + */ +class TabSeparatedRowOutputFormat : public IRowOutputFormat +{ +public: + /** with_names - output in the first line a header with column names + * with_types - output the next line header with the names of the types + */ + TabSeparatedRowOutputFormat(WriteBuffer & out, Block header, bool with_names, bool with_types, const FormatSettings & format_settings); + + String getName() const override { return "TabSeparatedRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowEndDelimiter() override; + void writePrefix() override; + void writeBeforeTotals() override; + void writeBeforeExtremes() override; + + /// https://www.iana.org/assignments/media-types/text/tab-separated-values + String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } + +protected: + + bool with_names; + bool with_types; + const FormatSettings format_settings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp new file mode 100644 index 00000000000..c7dd47831e6 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_READ_ARRAY_FROM_TEXT; + extern const int CANNOT_PARSE_DATE; + extern const int SYNTAX_ERROR; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} + + +ValuesRowInputFormat::ValuesRowInputFormat( + ReadBuffer & in, Block header, Params params, const Context & context_, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header), in, params) + , context(std::make_unique(context_)), format_settings(format_settings) +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); +} + + +bool ValuesRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + size_t num_columns = columns.size(); + auto & header = getPort().getHeader(); + + skipWhitespaceIfAny(in); + + if (in.eof() || *in.position() == ';') + return false; + + /** Typically, this is the usual format for streaming parsing. + * But as an exception, it also supports processing arbitrary expressions instead of values. + * This is very inefficient. But if there are no expressions, then there is no overhead. + */ + ParserExpression parser; + + assertChar('(', in); + + for (size_t i = 0; i < num_columns; ++i) + { + skipWhitespaceIfAny(in); + + char * prev_in_position = in.position(); + size_t prev_in_bytes = in.count() - in.offset(); + + bool rollback_on_exception = false; + try + { + header.getByPosition(i).type->deserializeAsTextQuoted(*columns[i], in, format_settings); + rollback_on_exception = true; + skipWhitespaceIfAny(in); + + if (i != num_columns - 1) + assertChar(',', in); + else + assertChar(')', in); + } + catch (const Exception & e) + { + if (!format_settings.values.interpret_expressions) + throw; + + /** The normal streaming parser could not parse the value. + * Let's try to parse it with a SQL parser as a constant expression. + * This is an exceptional case. + */ + if (e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED + || e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING + || e.code() == ErrorCodes::CANNOT_PARSE_NUMBER + || e.code() == ErrorCodes::CANNOT_PARSE_DATE + || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME + || e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT) + { + /// TODO Case when the expression does not fit entirely in the buffer. + + /// If the beginning of the value is no longer in the buffer. + if (in.count() - in.offset() != prev_in_bytes) + throw; + + if (rollback_on_exception) + columns[i]->popBack(1); + + const IDataType & type = *header.getByPosition(i).type; + + Expected expected; + + Tokens tokens(prev_in_position, in.buffer().end()); + TokenIterator token_iterator(tokens); + + ASTPtr ast; + if (!parser.parse(token_iterator, ast, expected)) + throw Exception("Cannot parse expression of type " + type.getName() + " here: " + + String(prev_in_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, in.buffer().end() - prev_in_position)), + ErrorCodes::SYNTAX_ERROR); + + in.position() = const_cast(token_iterator->begin); + + std::pair value_raw = evaluateConstantExpression(ast, *context); + Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); + + /// Check that we are indeed allowed to insert a NULL. + if (value.isNull()) + { + if (!type.isNullable()) + throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value) + + ", that is out of range of type " + type.getName() + + ", at: " + String(prev_in_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, in.buffer().end() - prev_in_position)), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + } + + columns[i]->insert(value); + + skipWhitespaceIfAny(in); + + if (i != num_columns - 1) + assertChar(',', in); + else + assertChar(')', in); + } + else + throw; + } + } + + skipWhitespaceIfAny(in); + if (!in.eof() && *in.position() == ',') + ++in.position(); + + return true; +} + + +void registerInputFormatProcessorValues(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("Values", []( + ReadBuffer & buf, + const Block & sample, + const Context & context, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, context, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.h new file mode 100644 index 00000000000..a0ff001c504 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class Context; +class ReadBuffer; + + +/** Stream to read data in VALUES format (as in INSERT query). + */ +class ValuesRowInputFormat : public IRowInputFormat +{ +public: + /** Data is parsed using fast, streaming parser. + * If interpret_expressions is true, it will, in addition, try to use SQL parser and interpreter + * in case when streaming parser could not parse field (this is very slow). + */ + ValuesRowInputFormat(ReadBuffer & in, Block header, Params params, const Context & context_, const FormatSettings & format_settings); + + String getName() const override { return "ValuesRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + +private: + std::unique_ptr context; /// pimpl + const FormatSettings format_settings; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp new file mode 100644 index 00000000000..ad9328ded95 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp @@ -0,0 +1,56 @@ +#include +#include + +#include +#include +#include + + +namespace DB +{ + + +ValuesRowOutputFormat::ValuesRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IRowOutputFormat(std::move(header), out), format_settings(format_settings) +{ +} + +void ValuesRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextQuoted(column, row_num, out, format_settings); +} + +void ValuesRowOutputFormat::writeFieldDelimiter() +{ + writeChar(',', out); +} + +void ValuesRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('(', out); +} + +void ValuesRowOutputFormat::writeRowEndDelimiter() +{ + writeChar(')', out); +} + +void ValuesRowOutputFormat::writeRowBetweenDelimiter() +{ + writeCString(",", out); +} + + +void registerOutputFormatProcessorValues(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Values", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h new file mode 100644 index 00000000000..f8b7a9f366d --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class WriteBuffer; + + +/** A stream for outputting data in the VALUES format (as in the INSERT request). + */ +class ValuesRowOutputFormat : public IRowOutputFormat +{ +public: + ValuesRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "ValuesRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; + +private: + const FormatSettings format_settings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp new file mode 100644 index 00000000000..1b8c6cf3bd8 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -0,0 +1,177 @@ +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +VerticalRowOutputFormat::VerticalRowOutputFormat( + WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IRowOutputFormat(std::move(header), out), format_settings(format_settings) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + size_t columns = sample.columns(); + + using Widths = std::vector; + Widths name_widths(columns); + size_t max_name_width = 0; + + String serialized_value; + + for (size_t i = 0; i < columns; ++i) + { + /// Note that number of code points is just a rough approximation of visible string width. + const String & name = sample.getByPosition(i).name; + + name_widths[i] = UTF8::computeWidth(reinterpret_cast(name.data()), name.size()); + + if (name_widths[i] > max_name_width) + max_name_width = name_widths[i]; + } + + names_and_paddings.resize(columns); + for (size_t i = 0; i < columns; ++i) + { + WriteBufferFromString out(names_and_paddings[i]); + writeString(sample.getByPosition(i).name, out); + writeCString(": ", out); + } + + for (size_t i = 0; i < columns; ++i) + { + size_t new_size = max_name_width - name_widths[i] + names_and_paddings[i].size(); + names_and_paddings[i].resize(new_size, ' '); + } +} + + +void VerticalRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + if (row_number > format_settings.pretty.max_rows) + return; + + writeString(names_and_paddings[field_number], out); + writeValue(column, type, row_num); + writeChar('\n', out); + + ++field_number; +} + + +void VerticalRowOutputFormat::writeValue(const IColumn & column, const IDataType & type, size_t row_num) const +{ + type.serializeAsText(column, row_num, out, format_settings); +} + + +void VerticalRowOutputFormat::writeRowStartDelimiter() +{ + ++row_number; + + if (row_number > format_settings.pretty.max_rows) + return; + + writeCString("Row ", out); + writeIntText(row_number, out); + writeCString(":\n", out); + + size_t width = log10(row_number + 1) + 1 + strlen("Row :"); + for (size_t i = 0; i < width; ++i) + writeCString("─", out); + writeChar('\n', out); +} + + +void VerticalRowOutputFormat::writeRowBetweenDelimiter() +{ + if (row_number > format_settings.pretty.max_rows) + return; + + writeCString("\n", out); + field_number = 0; +} + + +void VerticalRowOutputFormat::writeSuffix() +{ + if (row_number > format_settings.pretty.max_rows) + { + writeCString("Showed first ", out); + writeIntText(format_settings.pretty.max_rows, out); + writeCString(".\n", out); + } +} + +void VerticalRowOutputFormat::writeBeforeTotals() +{ + writeCString("\n", out); +} + +void VerticalRowOutputFormat::writeBeforeExtremes() +{ + if (!was_totals_written) + writeCString("\n", out); +} + +void VerticalRowOutputFormat::writeMinExtreme(const Columns & columns, size_t row_num) +{ + writeSpecialRow(columns, row_num, PortKind::Totals, "Min"); +} + +void VerticalRowOutputFormat::writeMaxExtreme(const Columns & columns, size_t row_num) +{ + writeSpecialRow(columns, row_num, PortKind::Totals, "Max"); +} + +void VerticalRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + writeSpecialRow(columns, row_num, PortKind::Totals, "Totals"); + was_totals_written = true; +} + +void VerticalRowOutputFormat::writeSpecialRow(const Columns & columns, size_t row_num, PortKind port_kind, const char * title) +{ + writeCString("\n", out); + + row_number = 0; + field_number = 0; + + auto & header = getPort(port_kind).getHeader(); + size_t num_columns = columns.size(); + + writeCString(title, out); + writeCString(":\n", out); + + size_t width = strlen(title) + 1; + for (size_t i = 0; i < width; ++i) + writeCString("─", out); + writeChar('\n', out); + + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + writeFieldDelimiter(); + + auto & col = header.getByPosition(i); + writeField(*columns[i], *col.type, row_num); + } +} + +void registerOutputFormatProcessorVertical(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Vertical", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h new file mode 100644 index 00000000000..b98bcb3924a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class WriteBuffer; +class Context; + + +/** Stream to output data in format "each value in separate row". + * Usable to show few rows with many columns. + */ +class VerticalRowOutputFormat : public IRowOutputFormat +{ +public: + VerticalRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "VerticalRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeRowStartDelimiter() override; + void writeRowBetweenDelimiter() override; + void writeSuffix() override; + + void writeMinExtreme(const Columns & columns, size_t row_num) override; + void writeMaxExtreme(const Columns & columns, size_t row_num) override; + void writeTotals(const Columns & columns, size_t row_num) override; + + void writeBeforeTotals() override; + void writeBeforeExtremes() override; + +protected: + virtual void writeValue(const IColumn & column, const IDataType & type, size_t row_num) const; + + /// For totals and extremes. + void writeSpecialRow(const Columns & columns, size_t row_num, PortKind port_kind, const char * title); + + const FormatSettings format_settings; + size_t field_number = 0; + size_t row_number = 0; + bool was_totals_written = false; + + using NamesAndPaddings = std::vector; + NamesAndPaddings names_and_paddings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp new file mode 100644 index 00000000000..3e38903fde7 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -0,0 +1,255 @@ +#include +#include +#include +#include + + +namespace DB +{ + +XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings) + : IRowOutputFormat(std::move(header), out), format_settings(format_settings) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); + field_tag_names.resize(sample.columns()); + + bool need_validate_utf8 = false; + for (size_t i = 0; i < sample.columns(); ++i) + { + if (!sample.getByPosition(i).type->textCanContainOnlyValidUTF8()) + need_validate_utf8 = true; + + /// As element names, we will use the column name if it has a valid form, or "field", otherwise. + /// The condition below is more strict than the XML standard requires. + bool is_column_name_suitable = true; + const char * begin = fields[i].name.data(); + const char * end = begin + fields[i].name.size(); + for (const char * pos = begin; pos != end; ++pos) + { + char c = *pos; + if (!(isAlphaASCII(c) + || (pos != begin && isNumericASCII(c)) + || c == '_' + || c == '-' + || c == '.')) + { + is_column_name_suitable = false; + break; + } + } + + field_tag_names[i] = is_column_name_suitable + ? fields[i].name + : "field"; + } + + if (need_validate_utf8) + { + validating_ostr = std::make_unique(out); + ostr = validating_ostr.get(); + } + else + ostr = &out; +} + + +void XMLRowOutputFormat::writePrefix() +{ + writeCString("\n", *ostr); + writeCString("\n", *ostr); + writeCString("\t\n", *ostr); + writeCString("\t\t\n", *ostr); + + for (const auto & field : fields) + { + writeCString("\t\t\t\n", *ostr); + + writeCString("\t\t\t\t", *ostr); + writeXMLString(field.name, *ostr); + writeCString("\n", *ostr); + writeCString("\t\t\t\t", *ostr); + writeXMLString(field.type->getName(), *ostr); + writeCString("\n", *ostr); + + writeCString("\t\t\t\n", *ostr); + } + + writeCString("\t\t\n", *ostr); + writeCString("\t\n", *ostr); + writeCString("\t\n", *ostr); +} + + +void XMLRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + writeCString("\t\t\t<", *ostr); + writeString(field_tag_names[field_number], *ostr); + writeCString(">", *ostr); + type.serializeAsTextXML(column, row_num, *ostr, format_settings); + writeCString("\n", *ostr); + ++field_number; +} + + +void XMLRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("\t\t\n", *ostr); +} + + +void XMLRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("\t\t\n", *ostr); + field_number = 0; + ++row_count; +} + + +void XMLRowOutputFormat::writeSuffix() +{ + writeCString("\t\n", *ostr); + +} + + +void XMLRowOutputFormat::writeBeforeTotals() +{ + writeCString("\t\n", *ostr); +} + +void XMLRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + size_t totals_columns = columns.size(); + auto & header = getPort(PortKind::Totals).getHeader(); + for (size_t i = 0; i < totals_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + writeCString("\t\t<", *ostr); + writeString(field_tag_names[i], *ostr); + writeCString(">", *ostr); + column.type->serializeAsTextXML(*columns[i], row_num, *ostr, format_settings); + writeCString("\n", *ostr); + } +} + +void XMLRowOutputFormat::writeAfterTotals() +{ + writeCString("\t\n", *ostr); +} + + +void XMLRowOutputFormat::writeBeforeExtremes() +{ + writeCString("\t\n", *ostr); +} + +void XMLRowOutputFormat::writeMinExtreme(const Columns & columns, size_t row_num) +{ + writeExtremesElement("min", columns, row_num); +} + +void XMLRowOutputFormat::writeMaxExtreme(const Columns & columns, size_t row_num) +{ + writeExtremesElement("max", columns, row_num); +} + +void XMLRowOutputFormat::writeAfterExtremes() +{ + writeCString("\t\n", *ostr); +} + +void XMLRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + auto & header = getPort(PortKind::Extremes).getHeader(); + + writeCString("\t\t<", *ostr); + writeCString(title, *ostr); + writeCString(">\n", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + writeCString("\t\t\t<", *ostr); + writeString(field_tag_names[i], *ostr); + writeCString(">", *ostr); + column.type->serializeAsTextXML(*columns[i], row_num, *ostr, format_settings); + writeCString("\n", *ostr); + } + + writeCString("\t\t\n", *ostr); +} + + +void XMLRowOutputFormat::onProgress(const Progress & value) +{ + progress.incrementPiecewiseAtomically(value); +} + +void XMLRowOutputFormat::writeLastSuffix() +{ + + writeCString("\t", *ostr); + writeIntText(row_count, *ostr); + writeCString("\n", *ostr); + + writeRowsBeforeLimitAtLeast(); + + if (format_settings.write_statistics) + writeStatistics(); + + writeCString("\n", *ostr); + ostr->next(); +} + +void XMLRowOutputFormat::writeRowsBeforeLimitAtLeast() +{ + if (applied_limit) + { + writeCString("\t", *ostr); + writeIntText(rows_before_limit, *ostr); + writeCString("\n", *ostr); + } +} + +void XMLRowOutputFormat::writeStatistics() +{ + writeCString("\t\n", *ostr); + writeCString("\t\t", *ostr); + writeText(watch.elapsedSeconds(), *ostr); + writeCString("\n", *ostr); + writeCString("\t\t", *ostr); + writeText(progress.rows.load(), *ostr); + writeCString("\n", *ostr); + writeCString("\t\t", *ostr); + writeText(progress.bytes.load(), *ostr); + writeCString("\n", *ostr); + writeCString("\t\n", *ostr); +} + + +void registerOutputFormatProcessorXML(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("XML", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h new file mode 100644 index 00000000000..73b05c61f60 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** A stream for outputting data in XML format. + */ +class XMLRowOutputFormat : public IRowOutputFormat +{ +public: + XMLRowOutputFormat(WriteBuffer & out, Block header, const FormatSettings & format_settings); + + String getName() const override { return "XMLRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void writePrefix() override; + void writeSuffix() override; + void writeLastSuffix() override; + + void writeMinExtreme(const Columns & columns, size_t row_num) override; + void writeMaxExtreme(const Columns & columns, size_t row_num) override; + void writeTotals(const Columns & columns, size_t row_num) override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + void writeBeforeExtremes() override; + void writeAfterExtremes() override; + + void flush() override + { + ostr->next(); + + if (validating_ostr) + out.next(); + } + + void setRowsBeforeLimit(size_t rows_before_limit_) override + { + applied_limit = true; + rows_before_limit = rows_before_limit_; + } + + void onProgress(const Progress & value) override; + + String getContentType() const override { return "application/xml; charset=UTF-8"; } + +protected: + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); + void writeRowsBeforeLimitAtLeast(); + void writeStatistics(); + + std::unique_ptr validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character. + WriteBuffer * ostr; + + size_t field_number = 0; + size_t row_count = 0; + bool applied_limit = false; + size_t rows_before_limit = 0; + NamesAndTypes fields; + Names field_tag_names; + + Progress progress; + Stopwatch watch; + const FormatSettings format_settings; +}; + +} + diff --git a/dbms/src/Processors/Formats/Impl/verbosePrintString.cpp b/dbms/src/Processors/Formats/Impl/verbosePrintString.cpp new file mode 100644 index 00000000000..1fdaf0cb7b7 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/verbosePrintString.cpp @@ -0,0 +1,64 @@ +#include +#include +#include + + +namespace DB +{ + +void verbosePrintString(const char * begin, const char * end, WriteBuffer & out) +{ + if (end == begin) + { + out << ""; + return; + } + + out << "\""; + + for (auto pos = begin; pos < end; ++pos) + { + switch (*pos) + { + case '\0': + out << ""; + break; + case '\b': + out << ""; + break; + case '\f': + out << "
"; + break; + case '\n': + out << ""; + break; + case '\r': + out << ""; + break; + case '\t': + out << ""; + break; + case '\\': + out << ""; + break; + case '"': + out << ""; + break; + case '\'': + out << ""; + break; + + default: + { + if (static_cast(*pos) < 32) /// ASCII control characters + out << "<0x" << hexDigitUppercase(*pos / 16) << hexDigitUppercase(*pos % 16) << ">"; + else + out << *pos; + } + } + } + + out << "\""; +} + +} diff --git a/dbms/src/Processors/Formats/Impl/verbosePrintString.h b/dbms/src/Processors/Formats/Impl/verbosePrintString.h new file mode 100644 index 00000000000..26bd663d559 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/verbosePrintString.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +class WriteBuffer; + + +/** Print string in double quotes and with control characters in "" form - for output diagnostic info to user. + */ +void verbosePrintString(const char * begin, const char * end, WriteBuffer & out); + +} diff --git a/dbms/src/Processors/ISource.h b/dbms/src/Processors/ISource.h index 9f07561311d..cf7bd8b771c 100644 --- a/dbms/src/Processors/ISource.h +++ b/dbms/src/Processors/ISource.h @@ -23,6 +23,7 @@ public: void work() override; OutputPort & getPort() { return output; } + const OutputPort & getPort() const { return output; } }; } diff --git a/dbms/src/Processors/Port.cpp b/dbms/src/Processors/Port.cpp index 9a867b8426a..deaeb124bdf 100644 --- a/dbms/src/Processors/Port.cpp +++ b/dbms/src/Processors/Port.cpp @@ -1,105 +1,8 @@ #include -#include namespace DB { -Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_) -{ - checkNumRowsIsConsistent(); -} - -Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(std::move(columns_)), num_rows(num_rows_), chunk_info(std::move(chunk_info_)) -{ - checkNumRowsIsConsistent(); -} - -static Columns unmuteColumns(MutableColumns && mut_columns) -{ - Columns columns; - columns.reserve(mut_columns.size()); - for (auto & col : mut_columns) - columns.emplace_back(std::move(col)); - - return columns; -} - -Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_) - : columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_) -{ -} - -Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_), chunk_info(std::move(chunk_info_)) -{ -} - -Chunk::Chunk(Chunk && other) noexcept - : columns(std::move(other.columns)) - , num_rows(other.num_rows) - , chunk_info(std::move(other.chunk_info)) -{ - other.num_rows = 0; -} - -Chunk & Chunk::operator=(Chunk && other) noexcept -{ - columns = std::move(other.columns); - chunk_info = std::move(other.chunk_info); - num_rows = other.num_rows; - other.num_rows = 0; - return *this; -} - -void Chunk::setColumns(Columns columns_, UInt64 num_rows_) -{ - columns = std::move(columns_); - num_rows = num_rows_; - checkNumRowsIsConsistent(); -} - -void Chunk::setColumns(MutableColumns columns_, UInt64 num_rows_) -{ - columns = unmuteColumns(std::move(columns_)); - num_rows = num_rows_; - checkNumRowsIsConsistent(); -} - -void Chunk::checkNumRowsIsConsistent() -{ - for (auto & column : columns) - if (column->size() != num_rows) - throw Exception("Invalid number of rows in Chunk column " + column->getName()+ ": expected " + - toString(num_rows) + ", got " + toString(column->size()), ErrorCodes::LOGICAL_ERROR); -} - -MutableColumns Chunk::mutateColumns() -{ - size_t num_columns = columns.size(); - MutableColumns mut_columns(num_columns); - for (size_t i = 0; i < num_columns; ++i) - mut_columns[i] = (*std::move(columns[i])).mutate(); - - columns.clear(); - num_rows = 0; - - return mut_columns; -} - -Columns Chunk::detachColumns() -{ - num_rows = 0; - return std::move(columns); -} - -void Chunk::clear() -{ - num_rows = 0; - columns.clear(); - chunk_info.reset(); -} - void connect(OutputPort & output, InputPort & input) { if (input.state || output.state) diff --git a/dbms/src/Processors/Port.h b/dbms/src/Processors/Port.h index ea98963ae57..b977a86e482 100644 --- a/dbms/src/Processors/Port.h +++ b/dbms/src/Processors/Port.h @@ -3,60 +3,12 @@ #include #include #include +#include #include namespace DB { -class ChunkInfo -{ -public: - virtual ~ChunkInfo() = default; -}; - -using ChunkInfoPtr = std::shared_ptr; - -class Chunk -{ -public: - Chunk() = default; - Chunk(const Chunk & other) = default; - Chunk(Chunk && other) noexcept; - Chunk(Columns columns_, UInt64 num_rows_); - Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); - Chunk(MutableColumns columns_, UInt64 num_rows_); - Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); - - Chunk & operator=(const Chunk & other) = default; - Chunk & operator=(Chunk && other) noexcept; - - const Columns & getColumns() { return columns; } - void setColumns(Columns columns_, UInt64 num_rows_); - void setColumns(MutableColumns columns_, UInt64 num_rows_); - Columns detachColumns(); - MutableColumns mutateColumns(); - - const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } - void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } - - UInt64 getNumRows() const { return num_rows; } - UInt64 getNumColumns() const { return columns.size(); } - bool empty() const { return num_rows == 0; } - operator bool() const { return !empty() || !columns.empty(); } - - void clear(); - -private: - Columns columns; - UInt64 num_rows = 0; - ChunkInfoPtr chunk_info; - - void checkNumRowsIsConsistent(); -}; - -using Chunks = std::vector; - - class InputPort; class OutputPort; class IProcessor;