From 49236beee87b4c1f4dc3cab842d2c68f3e9c0c4f Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 23 Dec 2015 11:11:11 +0300 Subject: [PATCH] dbms: CAST(expression AS `type`) [#METR-19265] --- dbms/include/DB/DataTypes/DataTypeEnum.h | 2 + .../DB/Functions/FunctionsConversion.h | 258 +++++++++++++++++- dbms/src/Functions/FunctionsConversion.cpp | 1 + dbms/src/Interpreters/ExpressionActions.cpp | 1 + dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 + dbms/src/Parsers/ExpressionElementParsers.cpp | 20 ++ 6 files changed, 281 insertions(+), 2 deletions(-) diff --git a/dbms/include/DB/DataTypes/DataTypeEnum.h b/dbms/include/DB/DataTypes/DataTypeEnum.h index 024826c71e9..159a44a093b 100644 --- a/dbms/include/DB/DataTypes/DataTypeEnum.h +++ b/dbms/include/DB/DataTypes/DataTypeEnum.h @@ -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 { diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index eeb1fc9e0b2..060673d8891 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -15,6 +16,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -102,7 +106,7 @@ struct ConvertImpl /// Реализация функции toDate. -namespace details { namespace { +namespace details { template class Transformation> class Transformer @@ -284,7 +288,7 @@ struct ToDateTransform32Or64 } }; -}} +} /** Преобразование даты-с-временем в дату: отбрасывание времени. */ @@ -1441,4 +1445,254 @@ typedef FunctionConvert FunctionToString; typedef FunctionConvert> 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(&*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( + column_const->size(), std::move(resized_string), new DataTypeFixedString(n)); + } + else if (const ColumnString * column_string = typeid_cast(&*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(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 wrapper_function; + + FunctionCast(const Context & context) : context(context) {} + + template + static auto createWrapper(const Context & context, const ColumnsWithTypeAndName & arguments) + { + std::shared_ptr function{static_cast(FunctionType::create(context))}; + + /// Check conversion using underlying function + DataTypePtr unused_type; + std::vector 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(from_type) && !typeid_cast(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(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(from_nested_type.get()); + to_type = typeid_cast(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 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(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(col_array->getDataPtr().get())) + { + /// create new level of array, copy offsets + res->getDataPtr() = new ColumnArray{nullptr, nested_col_array->getOffsetsColumn()}; + + res = static_cast(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(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 & 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(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(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (typeid_cast(type)) + wrapper_function = createWrapper(context, new_args); + else if (const auto type_fixed_string = typeid_cast(type)) + wrapper_function = createFixedStringWrapper(arguments[0].type.get(), type_fixed_string->getN()); + else if (const auto type_array = typeid_cast(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); + } +}; + } diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index e3f99046991..45b3139da3c 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -21,6 +21,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 9bdc2b5e81c..cbcb0c5f437 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -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) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index f4bcff6f234..e87a764cb56 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -45,6 +45,7 @@ #include #include +#include namespace DB diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index c67f5a5b567..d510e8c30e2 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -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;