From 84c23517745d207c68617beb0b13f5c5f97a6b15 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Tue, 26 May 2015 15:44:34 +0300 Subject: [PATCH 1/7] remove old todo --- dbms/src/Interpreters/InterpreterCheckQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.cpp b/dbms/src/Interpreters/InterpreterCheckQuery.cpp index 8bbe323c835..cc85f33c74c 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCheckQuery.cpp @@ -12,7 +12,6 @@ InterpreterCheckQuery::InterpreterCheckQuery(DB::ASTPtr query_ptr_, DB::Context& BlockInputStreamPtr InterpreterCheckQuery::execute() { - /// @TODO ASTCheckQuery & alter = typeid_cast(*query_ptr); String & table_name = alter.table; String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database; From bff45a25fad7dac0adf37f871a3ae16a6fc0396d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 02:09:56 +0300 Subject: [PATCH 2/7] dbms: improved type checking in IN [#METR-16793]. --- dbms/src/Interpreters/Set.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 8c4a9dc10f6..7f0fb917923 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -325,8 +325,21 @@ static Field convertToType(const Field & src, const IDataType & type) + Field::Types::toString(src.getType()) + " literal at right"); } } + else + { + if (src.getType() == Field::Types::UInt64 + || src.getType() == Field::Types::Int64 + || src.getType() == Field::Types::Float64 + || src.getType() == Field::Types::Null + || (src.getType() == Field::Types::String + && !typeid_cast(&type) + && !typeid_cast(&type)) + || (src.getType() == Field::Types::Array + && !typeid_cast(&type))) + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " + + Field::Types::toString(src.getType()) + " literal at right"); + } - /// В остальных случаях, приведение типа не осуществляется. return src; } From 5b8008e2b1506e29d877621abe9405156e83b4c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 02:38:55 +0300 Subject: [PATCH 3/7] dbms: fixed error with tuples [#METR-13649]. --- dbms/include/DB/DataTypes/DataTypeTuple.h | 1 + .../queries/0_stateless/00171_array_of_tuple_remote.reference | 2 ++ dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.sql | 1 + 3 files changed, 4 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.reference create mode 100644 dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.sql diff --git a/dbms/include/DB/DataTypes/DataTypeTuple.h b/dbms/include/DB/DataTypes/DataTypeTuple.h index 5b0b45db805..db9dd5bdb09 100644 --- a/dbms/include/DB/DataTypes/DataTypeTuple.h +++ b/dbms/include/DB/DataTypes/DataTypeTuple.h @@ -138,6 +138,7 @@ public: { ColumnWithNameAndType col; col.column = elems[i]->createColumn(); + col.type = elems[i]->clone(); tuple_block.insert(col); } return new ColumnTuple(tuple_block); diff --git a/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.reference b/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.reference new file mode 100644 index 00000000000..cd4823e219f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.reference @@ -0,0 +1,2 @@ +[(1,4),(2,5),(3,6)] +[(1,4),(2,5),(3,6)] diff --git a/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.sql b/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.sql new file mode 100644 index 00000000000..5db737e8e8b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00171_array_of_tuple_remote.sql @@ -0,0 +1 @@ +SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) FROM remote('127.0.0.{1,2}', system.one) ORDER BY rand(); From a9becd36ea876a2a5ae63ba37d0a73e2dbbfbd21 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 08:18:47 +0300 Subject: [PATCH 4/7] dbms: allowed constant expressions in IN [#METR-2944]. --- dbms/include/DB/Interpreters/Set.h | 2 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- dbms/src/Interpreters/Set.cpp | 76 +++++++++++++------- 3 files changed, 54 insertions(+), 26 deletions(-) diff --git a/dbms/include/DB/Interpreters/Set.h b/dbms/include/DB/Interpreters/Set.h index 4b6bfce1d75..e2dc2405279 100644 --- a/dbms/include/DB/Interpreters/Set.h +++ b/dbms/include/DB/Interpreters/Set.h @@ -283,7 +283,7 @@ public: * node - это список значений: 1, 2, 3 или список tuple-ов: (1, 2), (3, 4), (5, 6). * create_ordered_set - создавать ли вектор упорядоченных элементов. Нужен для работы индекса */ - void createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set); + void createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set); // Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять. bool insertFromBlock(const Block & block, bool create_ordered_set = false); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 9307b811854..8906e36fbfa 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -954,7 +954,7 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl ASTPtr ast_set_ptr = ast_set; ast_set->set = new Set(settings.limits); ast_set->is_explicit = true; - ast_set->set->createFromAST(set_element_types, elements_ast, create_ordered_set); + ast_set->set->createFromAST(set_element_types, elements_ast, context, create_ordered_set); arg = ast_set_ptr; } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 7f0fb917923..8e74ee231ef 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -10,8 +10,12 @@ #include #include #include +#include #include +#include +#include + #include #include #include @@ -259,10 +263,10 @@ static Field convertToType(const Field & src, const IDataType & type) if (is_uint8 || is_uint16 || is_uint32 || is_uint64) { if (src.getType() == Field::Types::Int64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed literal at right"); + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed at right"); if (src.getType() == Field::Types::Float64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point literal at right"); + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right"); if (src.getType() == Field::Types::UInt64) { @@ -276,12 +280,12 @@ static Field convertToType(const Field & src, const IDataType & type) } throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " literal at right"); + + Field::Types::toString(src.getType()) + " at right"); } else if (is_int8 || is_int16 || is_int32 || is_int64) { if (src.getType() == Field::Types::Float64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point literal at right"); + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right"); if (src.getType() == Field::Types::UInt64) { @@ -308,7 +312,7 @@ static Field convertToType(const Field & src, const IDataType & type) } throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " literal at right"); + + Field::Types::toString(src.getType()) + " at right"); } else if (is_float32 || is_float64) { @@ -322,7 +326,7 @@ static Field convertToType(const Field & src, const IDataType & type) return src; throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " literal at right"); + + Field::Types::toString(src.getType()) + " at right"); } } else @@ -337,22 +341,54 @@ static Field convertToType(const Field & src, const IDataType & type) || (src.getType() == Field::Types::Array && !typeid_cast(&type))) throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " literal at right"); + + Field::Types::toString(src.getType()) + " at right"); } return src; } -void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set) +/** Выполнить константное выражение (для элемента множества в IN). Весьма неоптимально. */ +static Field evaluateConstantExpression(ASTPtr & node, const Context & context) { - /** NOTE: - * На данный момент в секции IN не поддерживаются выражения (вызовы функций), кроме кортежей. - * То есть, например, не поддерживаются массивы. А по хорошему, хотелось бы поддерживать. - * Для этого можно сделать constant folding с помощью ExpressionAnalyzer/ExpressionActions. - * Но при этом, конечно же, не забыть про производительность работы с крупными множествами. - */ + ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer( + node, context, NamesAndTypesList{{ "_dummy", new DataTypeUInt8 }}).getConstActions(); + /// В блоке должен быть хотя бы один столбец, чтобы у него было известно число строк. + Block block_with_constants{{ new ColumnConstUInt8(1, 0), new DataTypeUInt8, "_dummy" }}; + + expr_for_constant_folding->execute(block_with_constants); + + if (!block_with_constants || block_with_constants.rows() == 0) + throw Exception("Logical error: empty block after evaluation constant expression for IN", ErrorCodes::LOGICAL_ERROR); + + String name = node->getColumnName(); + + if (!block_with_constants.has(name)) + throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS); + + const IColumn & result_column = *block_with_constants.getByName(name).column; + + if (!result_column.isConst()) + throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS); + + return result_column[0]; +} + + +static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context) +{ + if (ASTLiteral * lit = typeid_cast(node.get())) + return convertToType(lit->value, type); + else if (typeid_cast(node.get())) + return convertToType(evaluateConstantExpression(node, context), type); + else + throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET); +} + + +void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set) +{ data_types = types; /// Засунем множество в блок. @@ -372,10 +408,7 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set) { if (data_types.size() == 1) { - if (ASTLiteral * lit = typeid_cast(&**it)) - block.getByPosition(0).column->insert(convertToType(lit->value, *data_types[0])); - else - throw Exception("Incorrect element of set. Must be literal.", ErrorCodes::INCORRECT_ELEMENT_OF_SET); + block.getByPosition(0).column->insert(extractValueFromNode(*it, *data_types[0], context)); } else if (ASTFunction * func = typeid_cast(&**it)) { @@ -388,16 +421,11 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set) for (size_t j = 0; j < tuple_size; ++j) { - if (ASTLiteral * lit = typeid_cast(&*func->arguments->children[j])) - block.getByPosition(j).column->insert(convertToType(lit->value, *data_types[j])); - else - throw Exception("Incorrect element of tuple in set. Must be literal.", ErrorCodes::INCORRECT_ELEMENT_OF_SET); + block.getByPosition(j).column->insert(extractValueFromNode(func->arguments->children[j], *data_types[j], context)); } } else throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET); - - /// NOTE: Потом можно реализовать возможность задавать константные выражения в множествах. } if (create_ordered_set) From 779d85afc90fe44059fd68e9a7de11289e548901 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 08:19:18 +0300 Subject: [PATCH 5/7] dbms: addition to prev. revision [#METR-2944]. --- dbms/src/Interpreters/Set.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 8e74ee231ef..6bb858a5aaf 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include From 7b76ff3118d5dc5920ea792e3775f6ebaac91cd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 08:54:49 +0300 Subject: [PATCH 6/7] dbms: additions to prev. revision [#METR-2944]. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 26 ++++++++++++------- dbms/src/Interpreters/Set.cpp | 19 ++++++++++++-- .../00172_constexprs_in_set.reference | 6 +++++ .../0_stateless/00172_constexprs_in_set.sql | 6 +++++ 4 files changed, 46 insertions(+), 11 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00172_constexprs_in_set.reference create mode 100644 dbms/tests/queries/0_stateless/00172_constexprs_in_set.sql diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8906e36fbfa..5b4d58b65f8 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -922,16 +922,24 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl if (ASTFunction * set_func = typeid_cast(&*arg)) { - if (set_func->name != "tuple") - throw Exception("Incorrect type of 2nd argument for function " + node->name + ". Must be subquery or set of values.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - /// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2). - ASTFunction * any_element = typeid_cast(&*set_func->arguments->children.at(0)); - if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple")) - single_value = true; + if (set_func->name == "tuple") + { + /// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2). + ASTFunction * any_element = typeid_cast(&*set_func->arguments->children.at(0)); + if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple")) + single_value = true; + else + elements_ast = set_func->arguments; + } else - elements_ast = set_func->arguments; + { + if (set_element_types.size() >= 2) + throw Exception("Incorrect type of 2nd argument for function " + node->name + + ". Must be subquery or set of " + toString(set_element_types.size()) + "-element tuples.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + single_value = true; + } } else if (typeid_cast(&*arg)) { diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 6bb858a5aaf..08a6c6e0564 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include namespace DB @@ -234,7 +236,7 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set) */ static Field convertToType(const Field & src, const IDataType & type) { - if (type.behavesAsNumber()) + if (type.isNumeric()) { bool is_uint8 = false; bool is_uint16 = false; @@ -246,6 +248,8 @@ static Field convertToType(const Field & src, const IDataType & type) bool is_int64 = false; bool is_float32 = false; bool is_float64 = false; + bool is_date = false; + bool is_datetime = false; false || (is_uint8 = typeid_cast(&type)) @@ -257,7 +261,10 @@ static Field convertToType(const Field & src, const IDataType & type) || (is_int32 = typeid_cast(&type)) || (is_int64 = typeid_cast(&type)) || (is_float32 = typeid_cast(&type)) - || (is_float64 = typeid_cast(&type)); + || (is_float64 = typeid_cast(&type)) + || (is_date = typeid_cast(&type)) + || (is_datetime = typeid_cast(&type)) + ; if (is_uint8 || is_uint16 || is_uint32 || is_uint64) { @@ -327,6 +334,14 @@ static Field convertToType(const Field & src, const IDataType & type) throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " + Field::Types::toString(src.getType()) + " at right"); } + else if (is_date || is_datetime) + { + if (src.getType() != Field::Types::UInt64) + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " + + Field::Types::toString(src.getType()) + " at right"); + + return src; + } } else { diff --git a/dbms/tests/queries/0_stateless/00172_constexprs_in_set.reference b/dbms/tests/queries/0_stateless/00172_constexprs_in_set.reference new file mode 100644 index 00000000000..c06d3de5a56 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00172_constexprs_in_set.reference @@ -0,0 +1,6 @@ +14 3 +1 +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00172_constexprs_in_set.sql b/dbms/tests/queries/0_stateless/00172_constexprs_in_set.sql new file mode 100644 index 00000000000..3c438417053 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00172_constexprs_in_set.sql @@ -0,0 +1,6 @@ +SELECT sumIf(number, x), sum(x) FROM (SELECT number, number IN (0 + 1, 2 + 3, toUInt64(concat('8', ''))) AS x FROM system.numbers LIMIT 10); +SELECT toDate('2015-06-12') IN toDate('2015-06-12'); +SELECT toDate('2015-06-12') IN (toDate('2015-06-12')); +SELECT today() IN (toDate('2014-01-01'), toDate(now())); +SELECT - -1 IN (2 - 1); +SELECT - -1 IN (2 - 1, 3); From 26813667eb8248614c3d77e7e73c28b15a9d4c59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2015 10:46:58 +0300 Subject: [PATCH 7/7] dbms: allowed to compare Date and DateTime with constant string [#METR-2944]. --- .../DB/Functions/FunctionsComparison.h | 214 +++++++++++++----- dbms/src/Interpreters/ExpressionActions.cpp | 3 +- ...e_date_time_with_constant_string.reference | 80 +++++++ ...compare_date_time_with_constant_string.sql | 83 +++++++ 4 files changed, 317 insertions(+), 63 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.reference create mode 100644 dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.sql diff --git a/dbms/include/DB/Functions/FunctionsComparison.h b/dbms/include/DB/Functions/FunctionsComparison.h index d459055367b..a901f363342 100644 --- a/dbms/include/DB/Functions/FunctionsComparison.h +++ b/dbms/include/DB/Functions/FunctionsComparison.h @@ -26,6 +26,10 @@ namespace DB * - даты; * - даты-с-временем; * внутри каждой группы, но не из разных групп. + * + * Исключение: можно сравнивать дату и дату-с-временем с константной строкой. Пример: EventDate = '2015-01-01'. + * + * TODO Массивы, кортежи. */ /** Игнорируем warning о сравнении signed и unsigned. @@ -391,9 +395,9 @@ public: private: template - bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector * col_left) + bool executeNumRightType(Block & block, size_t result, const ColumnVector * col_left, const IColumn * col_right_untyped) { - if (ColumnVector * col_right = typeid_cast *>(&*block.getByPosition(arguments[1]).column)) + if (const ColumnVector * col_right = typeid_cast *>(col_right_untyped)) { ColumnUInt8 * col_res = new ColumnUInt8; block.getByPosition(result).column = col_res; @@ -404,7 +408,7 @@ private: return true; } - else if (ColumnConst * col_right = typeid_cast *>(&*block.getByPosition(arguments[1]).column)) + else if (const ColumnConst * col_right = typeid_cast *>(col_right_untyped)) { ColumnUInt8 * col_res = new ColumnUInt8; block.getByPosition(result).column = col_res; @@ -420,9 +424,9 @@ private: } template - bool executeNumConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst * col_left) + bool executeNumConstRightType(Block & block, size_t result, const ColumnConst * col_left, const IColumn * col_right_untyped) { - if (ColumnVector * col_right = typeid_cast *>(&*block.getByPosition(arguments[1]).column)) + if (const ColumnVector * col_right = typeid_cast *>(col_right_untyped)) { ColumnUInt8 * col_res = new ColumnUInt8; block.getByPosition(result).column = col_res; @@ -433,7 +437,7 @@ private: return true; } - else if (ColumnConst * col_right = typeid_cast *>(&*block.getByPosition(arguments[1]).column)) + else if (const ColumnConst * col_right = typeid_cast *>(col_right_untyped)) { UInt8 res = 0; NumComparisonImpl>::constant_constant(col_left->getData(), col_right->getData(), res); @@ -448,41 +452,41 @@ private: } template - bool executeNumLeftType(Block & block, const ColumnNumbers & arguments, size_t result) + bool executeNumLeftType(Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped) { - if (ColumnVector * col_left = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + if (const ColumnVector * col_left = typeid_cast *>(col_left_untyped)) { - if ( executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left) - || executeNumRightType(block, arguments, result, col_left)) + if ( executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped) + || executeNumRightType(block, result, col_left, col_right_untyped)) return true; else - throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + throw Exception("Illegal column " + col_right_untyped->getName() + " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } - else if (ColumnConst * col_left = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + else if (const ColumnConst * col_left = typeid_cast *>(col_left_untyped)) { - if ( executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left) - || executeNumConstRightType(block, arguments, result, col_left)) + if ( executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped) + || executeNumConstRightType(block, result, col_left, col_right_untyped)) return true; else - throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + throw Exception("Illegal column " + col_right_untyped->getName() + " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } @@ -490,17 +494,14 @@ private: return false; } - void executeString(Block & block, const ColumnNumbers & arguments, size_t result) + void executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1) { - IColumn * c0 = &*block.getByPosition(arguments[0]).column; - IColumn * c1 = &*block.getByPosition(arguments[1]).column; - - ColumnString * c0_string = typeid_cast(c0); - ColumnString * c1_string = typeid_cast(c1); - ColumnFixedString * c0_fixed_string = typeid_cast(c0); - ColumnFixedString * c1_fixed_string = typeid_cast(c1); - ColumnConstString * c0_const = typeid_cast(c0); - ColumnConstString * c1_const = typeid_cast(c1); + const ColumnString * c0_string = typeid_cast(c0); + const ColumnString * c1_string = typeid_cast(c1); + const ColumnFixedString * c0_fixed_string = typeid_cast(c0); + const ColumnFixedString * c1_fixed_string = typeid_cast(c1); + const ColumnConstString * c0_const = typeid_cast(c0); + const ColumnConstString * c1_const = typeid_cast(c1); using StringImpl = StringComparisonImpl>; @@ -559,13 +560,66 @@ private: c_res->getData()); else throw Exception("Illegal columns " - + block.getByPosition(arguments[0]).column->getName() + " and " - + block.getByPosition(arguments[1]).column->getName() + + c0->getName() + " and " + c1->getName() + " of arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } + void executeDateOrDateTimeWithConstString(Block & block, size_t result, + const IColumn * col_left_untyped, const IColumn * col_right_untyped, + bool left_is_num, bool right_is_num) + { + /// Особый случай - сравнение дат и дат-с-временем со строковой константой. + const IColumn * column_date_or_datetime = left_is_num ? col_left_untyped : col_right_untyped; + const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped; + + bool is_date = false; + bool is_date_time = false; + + is_date = typeid_cast *>(column_date_or_datetime) + || typeid_cast *>(column_date_or_datetime); + + if (!is_date) + is_date_time = typeid_cast *>(column_date_or_datetime) + || typeid_cast *>(column_date_or_datetime); + + const ColumnConstString * column_string = typeid_cast(column_string_untyped); + + if (!column_string + || (!is_date && !is_date_time)) + throw Exception("Illegal columns " + col_left_untyped->getName() + " and " + col_right_untyped->getName() + + " of arguments of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + if (is_date) + { + DayNum_t date; + ReadBufferFromString in(column_string->getData()); + readDateText(date, in); + if (!in.eof()) + throw Exception("String is too long for Date: " + column_string->getData()); + + ColumnConst parsed_const_date(block.rowsInFirstColumn(), date); + executeNumLeftType(block, result, + left_is_num ? col_left_untyped : &parsed_const_date, + left_is_num ? &parsed_const_date : col_right_untyped); + } + else if (is_date_time) + { + time_t date_time; + ReadBufferFromString in(column_string->getData()); + readDateTimeText(date_time, in); + if (!in.eof()) + throw Exception("String is too long for DateTime: " + column_string->getData()); + + ColumnConst parsed_const_date_time(block.rowsInFirstColumn(), date_time); + executeNumLeftType(block, result, + left_is_num ? col_left_untyped : &parsed_const_date_time, + left_is_num ? &parsed_const_date_time : col_right_untyped); + } + } + public: /// Получить имя функции. String getName() const @@ -581,12 +635,36 @@ public: + toString(arguments.size()) + ", should be 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!( ( arguments[0]->isNumeric() && arguments[0]->behavesAsNumber() - && arguments[1]->isNumeric() && arguments[1]->behavesAsNumber()) - || ( (arguments[0]->getName() == "String" || arguments[0]->getName().substr(0, 11) == "FixedString") - && (arguments[1]->getName() == "String" || arguments[1]->getName().substr(0, 11) == "FixedString")) - || (arguments[0]->getName() == "Date" && arguments[1]->getName() == "Date") - || (arguments[0]->getName() == "DateTime" && arguments[1]->getName() == "DateTime"))) + bool left_is_date = false; + bool left_is_date_time = false; + bool left_is_string = false; + bool left_is_fixed_string = false; + + false + || (left_is_date = typeid_cast(arguments[0].get())) + || (left_is_date_time = typeid_cast(arguments[0].get())) + || (left_is_string = typeid_cast(arguments[0].get())) + || (left_is_fixed_string = typeid_cast(arguments[0].get())); + + bool right_is_date = false; + bool right_is_date_time = false; + bool right_is_string = false; + bool right_is_fixed_string = false; + + false + || (right_is_date = typeid_cast(arguments[1].get())) + || (right_is_date_time = typeid_cast(arguments[1].get())) + || (right_is_string = typeid_cast(arguments[1].get())) + || (right_is_fixed_string = typeid_cast(arguments[1].get())); + + if (!( (arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber()) + || ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string)) + || (left_is_date && right_is_date) + || (left_is_date && right_is_string) /// Можно сравнивать дату и дату-с-временем с константной строкой. + || (left_is_string && right_is_date) + || (left_is_date_time && right_is_date_time) + || (left_is_date_time && right_is_string) + || (left_is_string && right_is_date_time))) throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")" " of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -596,24 +674,36 @@ public: /// Выполнить функцию над блоком. void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - if (block.getByPosition(arguments[0]).column->isNumeric()) + const IColumn * col_left_untyped = block.getByPosition(arguments[0]).column.get(); + const IColumn * col_right_untyped = block.getByPosition(arguments[1]).column.get(); + + bool left_is_num = col_left_untyped->isNumeric(); + bool right_is_num = col_right_untyped->isNumeric(); + + if (left_is_num && right_is_num) { - if (!( executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result) - || executeNumLeftType(block, arguments, result))) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + if (!( executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped))) + throw Exception("Illegal column " + col_left_untyped->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } + else if (!left_is_num && !right_is_num) + { + executeString(block, result, col_left_untyped, col_right_untyped); + } else - executeString(block, arguments, result); + { + executeDateOrDateTimeWithConstString(block, result, col_left_untyped, col_right_untyped, left_is_num, right_is_num); + } } }; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index fbfdd349381..c6b78cfbc5a 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -124,12 +124,13 @@ void ExpressionAction::prepare(Block & sample_block) /// Если все аргументы и требуемые столбцы - константы, выполним функцию. if (all_const) { + size_t result_position = sample_block.columns(); + ColumnWithNameAndType new_column; new_column.name = result_name; new_column.type = result_type; sample_block.insert(new_column); - size_t result_position = sample_block.getPositionByName(result_name); function->execute(sample_block, arguments, prerequisites, result_position); /// Если получилась не константа, на всякий случай будем считать результат неизвестным. diff --git a/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.reference b/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.reference new file mode 100644 index 00000000000..7f8dca8f200 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.reference @@ -0,0 +1,80 @@ +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +0 +0 +0 +1 +1 +0 +0 +1 +1 +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +0 +0 +0 +1 +1 +0 +0 +1 +1 +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +0 +0 +0 +1 +1 +0 +0 +1 +1 +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +0 +0 +0 +1 +1 +0 +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.sql b/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.sql new file mode 100644 index 00000000000..c89292a252a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00173_compare_date_time_with_constant_string.sql @@ -0,0 +1,83 @@ +SELECT toDate('2015-02-03') = '2015-02-03'; +SELECT '2015-02-03' = toDate('2015-02-03'); +SELECT toDate('2015-02-03') = '2015-02-04'; +SELECT '2015-02-03' = toDate('2015-02-04'); +SELECT toDate('2015-02-03') < '2015-02-04'; +SELECT '2015-02-03' < toDate('2015-02-04'); +SELECT toDate('2015-02-03') > '2015-02-04'; +SELECT '2015-02-03' > toDate('2015-02-04'); +SELECT toDate('2015-02-03') <= '2015-02-04'; +SELECT '2015-02-03' <= toDate('2015-02-04'); +SELECT toDate('2015-02-03') >= '2015-02-04'; +SELECT '2015-02-03' >= toDate('2015-02-04'); +SELECT toDate('2015-02-05') < '2015-02-04'; +SELECT '2015-02-05' < toDate('2015-02-04'); +SELECT toDate('2015-02-05') > '2015-02-04'; +SELECT '2015-02-05' > toDate('2015-02-04'); +SELECT toDate('2015-02-05') <= '2015-02-04'; +SELECT '2015-02-05' <= toDate('2015-02-04'); +SELECT toDate('2015-02-05') >= '2015-02-04'; +SELECT '2015-02-05' >= toDate('2015-02-04'); + +SELECT materialize(toDate('2015-02-03')) = '2015-02-03'; +SELECT '2015-02-03' = materialize(toDate('2015-02-03')); +SELECT materialize(toDate('2015-02-03')) = '2015-02-04'; +SELECT '2015-02-03' = materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-03')) < '2015-02-04'; +SELECT '2015-02-03' < materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-03')) > '2015-02-04'; +SELECT '2015-02-03' > materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-03')) <= '2015-02-04'; +SELECT '2015-02-03' <= materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-03')) >= '2015-02-04'; +SELECT '2015-02-03' >= materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-05')) < '2015-02-04'; +SELECT '2015-02-05' < materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-05')) > '2015-02-04'; +SELECT '2015-02-05' > materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-05')) <= '2015-02-04'; +SELECT '2015-02-05' <= materialize(toDate('2015-02-04')); +SELECT materialize(toDate('2015-02-05')) >= '2015-02-04'; +SELECT '2015-02-05' >= materialize(toDate('2015-02-04')); + +SELECT toDateTime('2015-02-03 04:05:06') = '2015-02-03 04:05:06'; +SELECT '2015-02-03 04:05:06' = toDateTime('2015-02-03 04:05:06'); +SELECT toDateTime('2015-02-03 04:05:06') = '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' = toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 04:05:06') < '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' < toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 04:05:06') > '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' > toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 04:05:06') <= '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' <= toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 04:05:06') >= '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' >= toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 06:07:08') < '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' < toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 06:07:08') > '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' > toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 06:07:08') <= '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' <= toDateTime('2015-02-03 05:06:07'); +SELECT toDateTime('2015-02-03 06:07:08') >= '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' >= toDateTime('2015-02-03 05:06:07'); + +SELECT materialize(toDateTime('2015-02-03 04:05:06')) = '2015-02-03 04:05:06'; +SELECT '2015-02-03 04:05:06' = materialize(toDateTime('2015-02-03 04:05:06')); +SELECT materialize(toDateTime('2015-02-03 04:05:06')) = '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' = materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 04:05:06')) < '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' < materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 04:05:06')) > '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' > materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 04:05:06')) <= '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' <= materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 04:05:06')) >= '2015-02-03 05:06:07'; +SELECT '2015-02-03 04:05:06' >= materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 06:07:08')) < '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' < materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 06:07:08')) > '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' > materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 06:07:08')) <= '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' <= materialize(toDateTime('2015-02-03 05:06:07')); +SELECT materialize(toDateTime('2015-02-03 06:07:08')) >= '2015-02-03 05:06:07'; +SELECT '2015-02-03 06:07:08' >= materialize(toDateTime('2015-02-03 05:06:07'));