mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
ClickHouse: added toFixedString function. [#METR-9091]
This commit is contained in:
parent
d2010d7876
commit
49a129c829
@ -11,6 +11,7 @@
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <Poco/NumberFormatter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -307,6 +308,105 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/** Преобразование в строку фиксированной длины реализовано только из строк.
|
||||
*/
|
||||
class FunctionToFixedString : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return "toFixedString";
|
||||
}
|
||||
|
||||
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
||||
* Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
* Для неконстантных столбцов arguments[i].column=NULL.
|
||||
*/
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
ExpressionActions::Actions & out_prerequisites)
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
if (!arguments[1].column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!dynamic_cast<const DataTypeString *>(&*arguments[0].type))
|
||||
throw Exception(getName() + " is only implemented for type String", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
size_t n = getSize(arguments[1]);
|
||||
|
||||
out_return_type = new DataTypeFixedString(n);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
size_t n = getSize(block.getByPosition(arguments[1]));
|
||||
|
||||
if (const ColumnConstString * column_const = dynamic_cast<const ColumnConstString *>(&*column))
|
||||
{
|
||||
if (column_const->getData().size() > n)
|
||||
throw Exception("String too long for type FixedString(" + Poco::NumberFormatter::format(n) + ")",
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
block.getByPosition(result).column = new ColumnConst<String>(column_const->size(), column_const->getData(), new DataTypeFixedString(n));
|
||||
}
|
||||
else if(const ColumnString * column_string = dynamic_cast<const ColumnString *>(&*column))
|
||||
{
|
||||
ColumnFixedString * column_fixed = new ColumnFixedString(n);
|
||||
ColumnPtr result_ptr = column_fixed;
|
||||
ColumnFixedString::Chars_t & out_chars = column_fixed->getChars();
|
||||
const ColumnString::Chars_t & in_chars = column_string->getChars();
|
||||
const ColumnString::Offsets_t & in_offsets = column_string->getOffsets();
|
||||
out_chars.resize_fill(in_offsets.size() * n);
|
||||
for (size_t i = 0; i < in_offsets.size(); ++i)
|
||||
{
|
||||
size_t off = i ? in_offsets[i - 1] : 0;
|
||||
size_t len = in_offsets[i] - off - 1;
|
||||
if (len > n)
|
||||
throw Exception("String too long for type FixedString(" + Poco::NumberFormatter::format(n) + ")",
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
memcpy(&out_chars[i * n], &in_chars[off], len);
|
||||
}
|
||||
block.getByPosition(result).column = result_ptr;
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool getSizeTyped(const ColumnWithNameAndType & column, size_t & out_size)
|
||||
{
|
||||
if (!dynamic_cast<const typename DataTypeFromFieldType<T>::Type *>(&*column.type))
|
||||
return false;
|
||||
const ColumnConst<T> * column_const = dynamic_cast<const ColumnConst<T> *>(&*column.column);
|
||||
if (!column_const)
|
||||
throw Exception("Unexpected type of column for FixedString length: " + column.column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
out_size = static_cast<size_t>(column_const->getData());
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getSize(const ColumnWithNameAndType & column)
|
||||
{
|
||||
size_t res;
|
||||
if (getSizeTyped<UInt8>(column, res) ||
|
||||
getSizeTyped<UInt16>(column, res) ||
|
||||
getSizeTyped<UInt32>(column, res) ||
|
||||
getSizeTyped<UInt64>(column, res) ||
|
||||
getSizeTyped< Int8 >(column, res) ||
|
||||
getSizeTyped< Int16>(column, res) ||
|
||||
getSizeTyped< Int32>(column, res) ||
|
||||
getSizeTyped< Int64>(column, res))
|
||||
return res;
|
||||
throw Exception("Length of FixedString must be integer; got " + column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameToUInt8 { static const char * get() { return "toUInt8"; } };
|
||||
struct NameToUInt16 { static const char * get() { return "toUInt16"; } };
|
||||
struct NameToUInt32 { static const char * get() { return "toUInt32"; } };
|
||||
|
@ -92,6 +92,7 @@ FunctionPtr FunctionFactory::get(
|
||||
else if (name == "toDate") return new FunctionToDate;
|
||||
else if (name == "toDateTime") return new FunctionToDateTime;
|
||||
else if (name == "toString") return new FunctionToString;
|
||||
else if (name == "toFixedString") return new FunctionToFixedString;
|
||||
|
||||
else if (name == "reinterpretAsUInt8") return new FunctionReinterpretAsUInt8;
|
||||
else if (name == "reinterpretAsUInt16") return new FunctionReinterpretAsUInt16;
|
||||
|
Loading…
Reference in New Issue
Block a user