mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
formatRow function
This commit is contained in:
parent
2bde393499
commit
b30240efc0
145
src/Functions/formatRow.cpp
Normal file
145
src/Functions/formatRow.cpp
Normal file
@ -0,0 +1,145 @@
|
||||
#include <memory>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
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(<format>, 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 <bool no_newline>
|
||||
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 <bool no_newline>
|
||||
class FormatRowOverloadResolver : public IFunctionOverloadResolverImpl
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow";
|
||||
static FunctionOverloadResolverImplPtr create(const Context & context) { return std::make_unique<FormatRowOverloadResolver>(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<ColumnString>(arguments.at(0).column.get()))
|
||||
return std::make_unique<DefaultFunction>(
|
||||
std::make_shared<FunctionFormatRow<no_newline>>(name_col->getValue<String>(), context),
|
||||
ext::map<DataTypes>(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<DataTypeString>(); }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
void registerFunctionFormatRow(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FormatRowOverloadResolver<true>>();
|
||||
factory.registerFunction<FormatRowOverloadResolver<false>>();
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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.
|
||||
*/
|
||||
|
12
tests/queries/0_stateless/01420_format_row.reference
Normal file
12
tests/queries/0_stateless/01420_format_row.reference
Normal file
@ -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"}
|
10
tests/queries/0_stateless/01420_format_row.sql
Normal file
10
tests/queries/0_stateless/01420_format_row.sql
Normal file
@ -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 }
|
Loading…
Reference in New Issue
Block a user