diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp new file mode 100644 index 00000000000..a923fa5cf73 --- /dev/null +++ b/src/Functions/formatRow.cpp @@ -0,0 +1,145 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; + extern const int UNKNOWN_FORMAT; +} + + +/** formatRow(, x, y, ...) is a function that allows you to use RowOutputFormat over + * several columns to generate a string per row, such as CSV, TSV, JSONEachRow, etc. + * formatRowNoNewline(...) trims the newline character of each row. + */ + +template +class FunctionFormatRow : public IFunction +{ +public: + static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; + + FunctionFormatRow(const String & format_name_, const Context & context_) : format_name(format_name_), context(context_) + { + if (!FormatFactory::instance().getAllFormats().count(format_name)) + throw Exception("Unknown format " + format_name, ErrorCodes::UNKNOWN_FORMAT); + + static constexpr auto supported_formats = {"CSV", "TSV", "JSONEachRow", "TabSeparated", "TabSeparatedRaw", "TSKV"}; + bool valid_format = false; + for (const auto & format : supported_formats) + { + if (format_name == format) + { + valid_format = true; + break; + } + } + if constexpr (!no_newline) + { + static constexpr auto supported_formats_no_newline = {"Values", "RowBinray"}; + for (const auto & format : supported_formats_no_newline) + { + if (format_name == format) + { + valid_format = true; + break; + } + } + } + if (!valid_format) + throw Exception("Function " + name + " doesn't support format " + format_name, ErrorCodes::NOT_IMPLEMENTED); + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto col_str = ColumnString::create(); + ColumnString::Chars & vec = col_str->getChars(); + WriteBufferFromVector buffer(vec); + ColumnString::Offsets & offsets = col_str->getOffsets(); + offsets.resize(input_rows_count); + Block arg_block; + for (auto i = 1u; i < arguments.size(); ++i) + arg_block.insert(block.getByPosition(arguments[i])); + materializeBlockInplace(arg_block); + auto out = FormatFactory::instance().getOutputFormat(format_name, buffer, arg_block, context, [&](const Columns &, size_t row) + { + if constexpr (no_newline) + *buffer.position() = '\0'; // replace '\n' with '\0' + else + writeChar('\0', buffer); + offsets[row] = buffer.count(); + }); + out->setAutoFlush(); + out->write(arg_block); + out->flush(); + block.getByPosition(result).column = std::move(col_str); + } + +private: + String format_name; + const Context & context; +}; + +template +class FormatRowOverloadResolver : public IFunctionOverloadResolverImpl +{ +public: + static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; + static FunctionOverloadResolverImplPtr create(const Context & context) { return std::make_unique(context); } + explicit FormatRowOverloadResolver(const Context & context_) : context(context_) { } + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + bool useDefaultImplementationForNulls() const override { return false; } + + FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (arguments.size() < 2) + throw Exception( + "Function " + getName() + " requires at least two arguments: the format name and its output expression(s)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (const auto * name_col = checkAndGetColumnConst(arguments.at(0).column.get())) + return std::make_unique( + std::make_shared>(name_col->getValue(), context), + ext::map(arguments, [](const auto & elem) { return elem.type; }), + return_type); + else + throw Exception("First argument to " + getName() + " must be a format name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + DataTypePtr getReturnType(const DataTypes &) const override { return std::make_shared(); } + +private: + const Context & context; +}; + +void registerFunctionFormatRow(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 5493b98d3fc..ae09498ee99 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -22,6 +22,7 @@ void registerFunctionReverse(FunctionFactory &); void registerFunctionReverseUTF8(FunctionFactory &); void registerFunctionsConcat(FunctionFactory &); void registerFunctionFormat(FunctionFactory &); +void registerFunctionFormatRow(FunctionFactory &); void registerFunctionSubstring(FunctionFactory &); void registerFunctionCRC(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); @@ -54,6 +55,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionReverseUTF8(factory); registerFunctionsConcat(factory); registerFunctionFormat(factory); + registerFunctionFormatRow(factory); registerFunctionSubstring(factory); registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index c68bc9438bf..047e16643ce 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -16,7 +16,7 @@ namespace ErrorCodes /** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size. * * In destructor, vector is cutted to the size of written data. - * You can call to 'finish' to resize earlier. + * You can call 'finalize' to resize earlier. * * The vector should live until this object is destroyed or until the 'finish' method is called. */ diff --git a/tests/queries/0_stateless/01420_format_row.reference b/tests/queries/0_stateless/01420_format_row.reference new file mode 100644 index 00000000000..95eaed7b156 --- /dev/null +++ b/tests/queries/0_stateless/01420_format_row.reference @@ -0,0 +1,12 @@ +0,"good"\n +1,"good"\n +2,"good"\n +0\t2001-12-12\t1.4 +1\t2001-12-12\t1.4 +2\t2001-12-12\t1.4 +{"number":"0","toNullable(3)":3,"NULL":null}\n +{"number":"1","toNullable(3)":3,"NULL":null}\n +{"number":"2","toNullable(3)":3,"NULL":null}\n +{"number":"0"} +{"number":"1"} +{"number":"2"} diff --git a/tests/queries/0_stateless/01420_format_row.sql b/tests/queries/0_stateless/01420_format_row.sql new file mode 100644 index 00000000000..634705208dc --- /dev/null +++ b/tests/queries/0_stateless/01420_format_row.sql @@ -0,0 +1,10 @@ +select formatRow('CSV', number, 'good') from numbers(3); +select formatRowNoNewline('TSV', number, DATE '2001-12-12', 1.4) from numbers(3); +select formatRow('JSONEachRow', number, toNullable(3), Null) from numbers(3); +select formatRowNoNewline('JSONEachRow', *) from numbers(3); + +-- unknown format +select formatRow('aaa', *) from numbers(3); -- { serverError 73 } + +-- unsupported format +select formatRowNoNewline('Values', *) from numbers(3); -- { serverError 48 }