dbms: CAST(expression AS type) [#METR-19265]

This commit is contained in:
Andrey Mironov 2015-12-23 11:11:11 +03:00
parent ce70e528e0
commit 49236beee8
6 changed files with 281 additions and 2 deletions

View File

@ -215,6 +215,8 @@ public:
x.resize(initial_size + size / sizeof(FieldType));
}
size_t getSizeOfField() const override { return sizeof(FieldType); }
ColumnPtr createColumn() const override { return new ColumnType; }
ColumnPtr createConstColumn(const size_t size, const Field & field) const override
{

View File

@ -2,6 +2,7 @@
#include <DB/IO/WriteBufferFromVector.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
@ -15,6 +16,9 @@
#include <DB/Core/FieldVisitors.h>
#include <ext/range.hpp>
#include <type_traits>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnArray.h>
namespace DB
@ -102,7 +106,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
/// Реализация функции toDate.
namespace details { namespace {
namespace details {
template<typename FromType, typename ToType, template <typename, typename> class Transformation>
class Transformer
@ -284,7 +288,7 @@ struct ToDateTransform32Or64
}
};
}}
}
/** Преобразование даты-с-временем в дату: отбрасывание времени.
*/
@ -1441,4 +1445,254 @@ typedef FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt
typedef FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity> FunctionToString;
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>> FunctionToUnixTimestamp;
struct CastToFixedStringImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
{
ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const auto column_const = typeid_cast<const ColumnConstString *>(&*column))
{
if (column_const->getData().size() > n)
throw Exception("String too long for type FixedString(" + toString(n) + ")",
ErrorCodes::TOO_LARGE_STRING_SIZE);
auto resized_string = column_const->getData();
resized_string.resize(n);
block.getByPosition(result).column = new ColumnConst<String>(
column_const->size(), std::move(resized_string), new DataTypeFixedString(n));
}
else if (const ColumnString * column_string = typeid_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(" + toString(n) + ")",
ErrorCodes::TOO_LARGE_STRING_SIZE);
memcpy(&out_chars[i * n], &in_chars[off], len);
}
block.getByPosition(result).column = result_ptr;
}
else if (const auto column_fixed_string = typeid_cast<const ColumnFixedString *>(column.get()))
{
const auto src_n = column_fixed_string->getN();
if (src_n > n)
throw Exception{
"String too long for type FixedString(" + toString(n) + ")",
ErrorCodes::TOO_LARGE_STRING_SIZE
};
const auto column_fixed = new ColumnFixedString{n};
block.getByPosition(result).column = column_fixed;
auto & out_chars = column_fixed->getChars();
const auto & in_chars = column_fixed_string->getChars();
const auto size = column_fixed_string->size();
out_chars.resize_fill(size * n);
for (const auto i : ext::range(0, size))
memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n);
}
else
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
class FunctionCast : public IFunction
{
const Context & context;
std::function<void(Block &, const ColumnNumbers &, size_t)> wrapper_function;
FunctionCast(const Context & context) : context(context) {}
template <typename FunctionType>
static auto createWrapper(const Context & context, const ColumnsWithTypeAndName & arguments)
{
std::shared_ptr<FunctionType> function{static_cast<FunctionType *>(FunctionType::create(context))};
/// Check conversion using underlying function
DataTypePtr unused_type;
std::vector<ExpressionAction> unused_prerequisites;
function->getReturnTypeAndPrerequisites(arguments, unused_type, unused_prerequisites);
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result) {
/// drop second argument, pass others
ColumnNumbers new_args{arguments.front()};
if (arguments.size() > 2)
new_args.insert(std::end(new_args), std::next(std::begin(arguments), 2), std::end(arguments));
function->execute(block, new_args, result);
};
}
static auto createFixedStringWrapper(const IDataType * from_type, const size_t N)
{
if (!typeid_cast<const DataTypeString *>(from_type) && !typeid_cast<const DataTypeFixedString *>(from_type))
throw Exception{
"CAST AS FixedString is only implemented for types String and FixedString",
ErrorCodes::NOT_IMPLEMENTED
};
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result)
{
CastToFixedStringImpl::execute(block, arguments, result, N);
};
}
auto createArrayWrapper(const IDataType * const from_type_untyped, const DataTypeArray * to_type)
{
DataTypePtr from_nested_type, to_nested_type;
auto from_type = typeid_cast<const DataTypeArray *>(from_type_untyped);
/// get the most nested type
while (from_type && to_type)
{
from_nested_type = from_type->getNestedType();
to_nested_type = to_type->getNestedType();
from_type = typeid_cast<const DataTypeArray *>(from_nested_type.get());
to_type = typeid_cast<const DataTypeArray *>(to_nested_type.get());
}
/// both from_type and to_type should be nullptr now is array types had same dimensions
if (from_type || to_type)
throw Exception{
"CAST AS Array can only convert between same-dimensional array types",
ErrorCodes::TYPE_MISMATCH
};
/// check that conversion between nested types is valid
DataTypePtr unused_type;
std::vector<ExpressionAction> unused_prerequisites;
const ColumnsWithTypeAndName nested_args{
{ nullptr, from_nested_type, "" },
{ new ColumnConstString{0, to_nested_type->getName()}, new DataTypeString, "" }
};
getReturnTypeAndPrerequisites(nested_args, unused_type, unused_prerequisites);
auto nested_function = wrapper_function;
return [nested_function, from_nested_type, to_nested_type] (
Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// @todo add const variant which retains array constness
const auto array_arg = block.getByPosition(arguments[0]);
if (auto col_array = typeid_cast<const ColumnArray *>(array_arg.column.get()))
{
auto res = new ColumnArray{nullptr, col_array->getOffsetsColumn()};
block.getByPosition(result).column = res;
/// get the most nested column
while (const auto nested_col_array = typeid_cast<const ColumnArray *>(col_array->getDataPtr().get()))
{
/// create new level of array, copy offsets
res->getDataPtr() = new ColumnArray{nullptr, nested_col_array->getOffsetsColumn()};
res = static_cast<ColumnArray *>(res->getDataPtr().get());
col_array = nested_col_array;
}
/// create block for converting nested column containing original and result columns
Block nested_block{
{ col_array->getDataPtr(), from_nested_type, "" },
{ nullptr, to_nested_type, "" }
};
const auto nested_result = 1;
/// convert nested column
nested_function(nested_block, {0 }, nested_result);
/// set converted nested column to result
res->getDataPtr() = nested_block.getByPosition(nested_result).column;
}
else if (const auto col_const_array = typeid_cast<const ColumnConstArray *>(array_arg.column.get()))
throw Exception{"NYI", ErrorCodes::NOT_IMPLEMENTED};
// column_array = col_const_array->convertToFullColumn();
else
throw Exception{
"Illegal column " + array_arg.column->getName() + " for function CAST AS Array",
ErrorCodes::LOGICAL_ERROR
};
};
}
public:
static constexpr auto name = "CAST";
static IFunction * create(const Context & context) { return new FunctionCast{context}; }
String getName() const override { return name; }
void getReturnTypeAndPrerequisites(
const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites) override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto type_col = typeid_cast<const ColumnConstString *>(arguments.back().column.get());
if (!type_col)
throw Exception("Second argument to " + getName() + " must be a constant string describing type",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto & type_name = type_col->getData();
out_return_type = DataTypeFactory::instance().get(type_name);
const auto type = out_return_type.get();
ColumnsWithTypeAndName new_args{arguments.front()};
if (typeid_cast<const DataTypeUInt8 *>(type))
wrapper_function = createWrapper<FunctionToUInt8>(context, new_args);
else if (typeid_cast<const DataTypeUInt16 *>(type))
wrapper_function = createWrapper<FunctionToUInt16>(context, new_args);
else if (typeid_cast<const DataTypeUInt32 *>(type))
wrapper_function = createWrapper<FunctionToUInt32>(context, new_args);
else if (typeid_cast<const DataTypeUInt64 *>(type))
wrapper_function = createWrapper<FunctionToUInt64>(context, new_args);
else if (typeid_cast<const DataTypeInt8 *>(type))
wrapper_function = createWrapper<FunctionToInt8>(context, new_args);
else if (typeid_cast<const DataTypeInt16 *>(type))
wrapper_function = createWrapper<FunctionToInt16>(context, new_args);
else if (typeid_cast<const DataTypeInt32 *>(type))
wrapper_function = createWrapper<FunctionToInt32>(context, new_args);
else if (typeid_cast<const DataTypeInt64 *>(type))
wrapper_function = createWrapper<FunctionToInt64>(context, new_args);
else if (typeid_cast<const DataTypeFloat32 *>(type))
wrapper_function = createWrapper<FunctionToFloat32>(context, new_args);
else if (typeid_cast<const DataTypeFloat64 *>(type))
wrapper_function = createWrapper<FunctionToFloat64>(context, new_args);
else if (typeid_cast<const DataTypeDate *>(type))
wrapper_function = createWrapper<FunctionToDate>(context, new_args);
else if (typeid_cast<const DataTypeDateTime *>(type))
wrapper_function = createWrapper<FunctionToDateTime>(context, new_args);
else if (typeid_cast<const DataTypeString *>(type))
wrapper_function = createWrapper<FunctionToString>(context, new_args);
else if (const auto type_fixed_string = typeid_cast<const DataTypeFixedString *>(type))
wrapper_function = createFixedStringWrapper(arguments[0].type.get(), type_fixed_string->getN());
else if (const auto type_array = typeid_cast<const DataTypeArray *>(type))
wrapper_function = createArrayWrapper(arguments[0].type.get(), type_array);
else
throw Exception{
"Not yet implemented CAST for type " + type_name,
ErrorCodes::NOT_IMPLEMENTED
};
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
{
wrapper_function(block, arguments, result);
}
};
}

View File

@ -21,6 +21,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToString>();
factory.registerFunction<FunctionToFixedString>();
factory.registerFunction<FunctionToUnixTimestamp>();
factory.registerFunction<FunctionCast>();
}
}

View File

@ -618,6 +618,7 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum
if (it == columns.end())
throw Exception("No available columns", ErrorCodes::LOGICAL_ERROR);
/// @todo resolve evil constant
size_t min_size = it->type->isNumeric() ? it->type->getSizeOfField() : 100;
String res = it->name;
for (; it != columns.end(); ++it)

View File

@ -45,6 +45,7 @@
#include <DB/Functions/FunctionFactory.h>
#include <ext/range.hpp>
#include <DB/DataTypes/DataTypeFactory.h>
namespace DB

View File

@ -265,6 +265,26 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
function_node->arguments = expr_list_args;
function_node->children.push_back(function_node->arguments);
if (function_node->name == "CAST")
{
/// Convert CAST(expression AS type) to CAST(expression, 'type')
if (expr_list_args->children.size() == 1)
{
const auto alias = expr_list_args->children.front()->tryGetAlias();
if (alias.empty())
throw Exception{
"CAST expression has to be in form CAST(expression AS type)",
ErrorCodes::SYNTAX_ERROR
};
expr_list_args->children.emplace_back(
new ASTLiteral{{}, alias}
);
expr_list_args->children.front()->setAlias({});
}
}
if (expr_list_params)
{
function_node->parameters = expr_list_params;