dbms: allow type in CAST(expression AS type) without back quotes [#METR-19265]

This commit is contained in:
Andrey Mironov 2015-12-24 20:14:10 +03:00
parent 0dc042536c
commit 40f060fae8
11 changed files with 256 additions and 120 deletions

View File

@ -220,7 +220,7 @@ public:
ColumnPtr createColumn() const override { return new ColumnType; }
ColumnPtr createConstColumn(const size_t size, const Field & field) const override
{
return new ColumnConst<FieldType>(size, get<typename NearestFieldType<FieldType>::Type>(field));;
return new ColumnConst<FieldType>(size, get<typename NearestFieldType<FieldType>::Type>(field));
}
Field getDefault() const override

View File

@ -1222,12 +1222,17 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
ColumnPtr column = block.getByPosition(arguments[0]).column;
size_t n = getSize(block.getByPosition(arguments[1]));
const auto n = getSize(block.getByPosition(arguments[1]));
return execute(block, arguments, result, n);
}
if (const ColumnConstString * column_const = typeid_cast<const ColumnConstString *>(&*column))
static void execute(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
{
const auto & 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) + ")",
@ -1236,25 +1241,31 @@ public:
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));
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))
else if (const auto column_string = typeid_cast<const ColumnString *>(&*column))
{
ColumnFixedString * column_fixed = new ColumnFixedString(n);
const auto 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();
auto & out_chars = column_fixed->getChars();
const auto & in_chars = column_string->getChars();
const auto & 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;
const size_t off = i ? in_offsets[i - 1] : 0;
const 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()))
@ -1449,67 +1460,6 @@ 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
{
@ -1547,7 +1497,7 @@ class FunctionCast : public IFunction
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result)
{
CastToFixedStringImpl::execute(block, arguments, result, N);
FunctionToFixedString::execute(block, arguments, result, N);
};
}
@ -1579,8 +1529,11 @@ class FunctionCast : public IFunction
return [nested_function, from_nested_type, to_nested_type] (
Block & block, const ColumnNumbers & arguments, const size_t result)
{
auto array_arg = block.getByPosition(arguments[0]);
/// @todo add const variant which retains array constness
const auto array_arg = block.getByPosition(arguments[0]);
if (const auto col_const_array = typeid_cast<const ColumnConstArray *>(array_arg.column.get()))
array_arg.column = col_const_array->convertToFullColumn();
if (auto col_array = typeid_cast<const ColumnArray *>(array_arg.column.get()))
{
@ -1610,9 +1563,6 @@ class FunctionCast : public IFunction
/// 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",

View File

@ -69,6 +69,16 @@ protected:
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
class ParserCastExpression : public IParserBase
{
/// this name is used for identifying CAST expression among other function calls
static constexpr auto name = "CAST";
protected:
const char * getName() const { return name; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) override;
};
/** NULL.
*/
@ -135,20 +145,34 @@ protected:
/** Алиас - идентификатор, перед которым идёт AS. Например: AS x_yz123.
*/
class ParserAlias : public IParserBase
struct ParserAliasBase
{
static const char * restricted_keywords[];
};
template <typename ParserIdentifier>
class ParserAliasImpl : public IParserBase, ParserAliasBase
{
public:
ParserAlias(bool allow_alias_without_as_keyword_)
ParserAliasImpl(bool allow_alias_without_as_keyword_)
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {}
protected:
bool allow_alias_without_as_keyword;
static const char * restricted_keywords[];
const char * getName() const { return "alias"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
class ParserTypeInCastExpression;
extern template class ParserAliasImpl<ParserIdentifier>;
extern template class ParserAliasImpl<ParserTypeInCastExpression>;
using ParserAlias = ParserAliasImpl<ParserIdentifier>;
using ParserCastExpressionAlias = ParserAliasImpl<ParserTypeInCastExpression>;
/** Элемент выражения - одно из: выражение в круглых скобках, массив, литерал, функция, идентификатор, звёздочка.
*/
class ParserExpressionElement : public IParserBase
@ -161,10 +185,11 @@ protected:
/** Элемент выражения, возможно, с алиасом, если уместно.
*/
class ParserWithOptionalAlias : public IParserBase
template <typename ParserAlias>
class ParserWithOptionalAliasImpl : public IParserBase
{
public:
ParserWithOptionalAlias(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_)
ParserWithOptionalAliasImpl(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_)
: elem_parser(std::move(elem_parser_)), allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {}
protected:
ParserPtr elem_parser;
@ -174,6 +199,12 @@ protected:
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
extern template class ParserWithOptionalAliasImpl<ParserAlias>;
extern template class ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
using ParserWithOptionalAlias = ParserWithOptionalAliasImpl<ParserAlias>;
using ParserCastExpressionWithOptionalAlias = ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
/** Элемент выражения ORDER BY - то же самое, что и элемент выражения, но после него ещё может быть указано ASC[ENDING] | DESC[ENDING]
* и, возможно, COLLATE 'locale'.

View File

@ -294,6 +294,21 @@ protected:
}
};
class ParserExpressionInCastExpression : public IParserBase
{
public:
ParserExpressionInCastExpression(bool allow_alias_without_as_keyword);
protected:
ParserPtr impl;
const char * getName() const { return "expression in CAST expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return impl->parse(pos, end, node, max_parsed_pos, expected);
}
};
/** Список выражений, разделённых запятыми, возможно пустой. */
class ParserExpressionList : public IParserBase

View File

@ -47,6 +47,13 @@ protected:
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
class ParserTypeInCastExpression : public ParserIdentifierWithOptionalParameters
{
protected:
const char * getName() const { return "type in cast expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
template <class NameParser>
class IParserNameTypePair : public IParserBase

View File

@ -170,14 +170,10 @@ DataTypePtr DataTypeFactory::get(const String & name) const
ParserExpressionList columns_p(false);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
DataTypes elems;
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_ast);
for (ASTs::iterator it = columns_list.children.begin(); it != columns_list.children.end(); ++it)
{
StringRange range = (*it)->range;
elems.push_back(get(String(range.first, range.second - range.first)));
}
auto & columns_list = typeid_cast<ASTExpressionList &>(*columns_ast);
const auto elems = ext::map<DataTypes>(columns_list.children, [this] (const ASTPtr & elem_ast) {
return get(String(elem_ast->range.first, elem_ast->range.second));
});
return new DataTypeTuple(elems);
}

View File

@ -16,6 +16,27 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
bool written = false;
if (arguments && !parameters)
{
if (0 == strcmp(name.data(), "CAST"))
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << name;
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
arguments->children.front()->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS "
<< (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "")
<< typeid_cast<const ASTLiteral &>(*arguments->children.back()).value.safeGet<String>()
<< (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << ')'
<< (settings.hilite ? hilite_none : "");
written = true;
}
if (arguments->children.size() == 1)
{
const char * operators[] =

View File

@ -18,7 +18,7 @@
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/ParserCreateQuery.h>
namespace DB
@ -260,38 +260,124 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
}
ASTFunction * function_node = new ASTFunction(StringRange(begin, pos));
ASTPtr node_holder{function_node};
function_node->name = typeid_cast<ASTIdentifier &>(*identifier).name;
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;
function_node->children.push_back(function_node->parameters);
}
node = function_node;
node = node_holder;
return true;
}
bool ParserCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
const auto begin = pos;
ParserIdentifier id_parser;
ASTPtr identifier;
if (!id_parser.parse(pos, end, identifier, max_parsed_pos, expected))
return false;
const auto & id = typeid_cast<const ASTIdentifier &>(*identifier).name;
if (id.length() != strlen(name) || 0 != strcasecmp(id.c_str(), name))
/// Parse as a simple ASTFunction
return ParserFunction{}.parse(pos = begin, end, node, max_parsed_pos, expected);
/// Parse as CAST(expression AS type)
ParserString open("("), close(")"), comma(",");
ParserExpressionInCastExpression expression_and_type(false);
ParserWhiteSpaceOrComments ws;
ASTPtr expr_list_args;
ws.ignore(pos, end);
if (!open.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
const auto contents_begin = pos;
ASTPtr first_argument;
if (!expression_and_type.parse(pos, end, first_argument, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
/// check for subsequent comma ","
if (!comma.ignore(pos, end, max_parsed_pos, expected))
{
/// CAST(expression AS type)
const auto type = first_argument->tryGetAlias();
if (type.empty())
{
/// there is only one argument and it has no alias
expected = "type identifier";
return false;
}
expr_list_args = new ASTExpressionList{StringRange{contents_begin, end}};
first_argument->setAlias({});
expr_list_args->children.push_back(first_argument);
expr_list_args->children.emplace_back(new ASTLiteral{{}, type});
}
else
{
/// CAST(expression, 'type')
/// Reparse argument list from scratch
max_parsed_pos = pos = contents_begin;
ParserExpressionWithOptionalAlias expression{false};
if (!expression.parse(pos, end, first_argument, max_parsed_pos, expected))
return false;
ws.ignore(pos, end, max_parsed_pos, expected);
if (!comma.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end, max_parsed_pos, expected);
ParserStringLiteral p_type;
ASTPtr type_as_literal;
if (!p_type.parse(pos, end, type_as_literal, max_parsed_pos, expected))
{
expected = "string literal depicting type";
return false;
}
expr_list_args = new ASTExpressionList{StringRange{contents_begin, end}};
expr_list_args->children.push_back(first_argument);
expr_list_args->children.push_back(type_as_literal);
}
ws.ignore(pos, end);
if (!close.ignore(pos, end, max_parsed_pos, expected))
{
expected = ")";
return false;
}
const auto function_node = new ASTFunction(StringRange(begin, pos));
ASTPtr node_holder{function_node};
function_node->name = name;
function_node->arguments = expr_list_args;
function_node->children.push_back(function_node->arguments);
node = node_holder;
return true;
}
@ -486,7 +572,7 @@ bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parse
}
const char * ParserAlias::restricted_keywords[] =
const char * ParserAliasBase::restricted_keywords[] =
{
"FROM",
"FINAL",
@ -516,7 +602,8 @@ const char * ParserAlias::restricted_keywords[] =
nullptr
};
bool ParserAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
template <typename ParserIdentifier>
bool ParserAliasImpl<ParserIdentifier>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserWhiteSpaceOrComments ws;
ParserString s_as("AS", true, true);
@ -548,6 +635,9 @@ bool ParserAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
return true;
}
template class ParserAliasImpl<ParserIdentifier>;
template class ParserAliasImpl<ParserTypeInCastExpression>;
bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
@ -558,7 +648,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
ParserArray array_p;
ParserArrayOfLiterals array_lite_p;
ParserLiteral lit_p;
ParserFunction fun_p;
ParserCastExpression fun_p;
ParserCompoundIdentifier id_p;
ParserString asterisk_p("*");
@ -589,12 +679,14 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
return true;
}
expected = "expression element: one of array, literal, function, identifier, asterisk, parenthised expression, subquery";
if (!expected)
expected = "expression element: one of array, literal, function, identifier, asterisk, parenthesised expression, subquery";
return false;
}
bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
template <typename ParserAlias>
bool ParserWithOptionalAliasImpl<ParserAlias>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserWhiteSpaceOrComments ws;
@ -645,6 +737,9 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
return true;
}
template class ParserWithOptionalAliasImpl<ParserAlias>;
template class ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{

View File

@ -6,6 +6,7 @@
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/Parsers/ParserCreateQuery.h>
namespace DB
@ -491,6 +492,12 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_
}
ParserExpressionInCastExpression::ParserExpressionInCastExpression(bool allow_alias_without_as_keyword)
: impl(new ParserCastExpressionWithOptionalAlias(ParserPtr(new ParserLambdaExpression), allow_alias_without_as_keyword))
{
}
bool ParserExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return ParserList(ParserPtr(new ParserExpressionWithOptionalAlias(allow_alias_without_as_keyword)), ParserPtr(new ParserString(","))).parse(pos, end, node, max_parsed_pos, expected);

View File

@ -88,6 +88,20 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPt
return false;
}
bool ParserTypeInCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
if (ParserIdentifierWithOptionalParameters::parseImpl(pos, end, node, max_parsed_pos, expected))
{
const auto & id_with_params = typeid_cast<const ASTFunction &>(*node);
node = new ASTIdentifier{id_with_params.range, { id_with_params.range.first, id_with_params.range.second }};
return true;
}
return false;
}
bool ParserNameTypePairList::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return ParserList(ParserPtr(new ParserNameTypePair), ParserPtr(new ParserString(",")), false).parse(pos, end, node, max_parsed_pos, expected);

View File

@ -31,7 +31,7 @@ key UInt64 MATERIALIZED 0 * rand()
hello clickhouse 16
some string 11
payload String
payload_length UInt16 DEFAULT toUInt16(length(payload))
payload_length UInt16 DEFAULT CAST(length(payload) AS UInt16)
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload String