From 77fd060665751fc6528dd9f77e0fdea41cbc23bc Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 14 Feb 2021 19:09:36 +0800 Subject: [PATCH 01/20] Normalize function names --- .../AggregateFunctionFactory.cpp | 16 +++-- src/Common/IFactoryWithAliases.h | 14 ++++ src/Functions/FunctionFactory.cpp | 15 +++-- src/Functions/FunctionsRound.cpp | 2 +- src/Functions/extractAllGroupsVertical.cpp | 2 +- src/Interpreters/FunctionNameNormalizer.cpp | 18 +++++ src/Interpreters/FunctionNameNormalizer.h | 14 ++++ src/Interpreters/MutationsInterpreter.cpp | 4 +- src/Interpreters/TreeRewriter.cpp | 4 ++ src/Interpreters/addTypeConversionToAST.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 2 +- .../Impl/ConstantExpressionTemplate.cpp | 2 +- tests/integration/test_mysql_protocol/test.py | 2 +- .../00597_push_down_predicate.reference | 2 +- .../01029_early_constant_folding.reference | 2 +- ...1611_constant_folding_subqueries.reference | 2 +- ..._case_insensitive_function_names.reference | 66 +++++++++++++++++++ ...malize_case_insensitive_function_names.sql | 1 + 18 files changed, 151 insertions(+), 19 deletions(-) create mode 100644 src/Interpreters/FunctionNameNormalizer.cpp create mode 100644 src/Interpreters/FunctionNameNormalizer.h create mode 100644 tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference create mode 100644 tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 5fc690d59f2..061077dd8fa 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -30,6 +30,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +const String & getAggregateFunctionCanonicalNameIfAny(const String & name) +{ + return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name); +} void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness) { @@ -41,10 +45,14 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat throw Exception("AggregateFunctionFactory: the aggregate function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - if (case_sensitiveness == CaseInsensitive - && !case_insensitive_aggregate_functions.emplace(Poco::toLower(name), creator_with_properties).second) - throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique", - ErrorCodes::LOGICAL_ERROR); + if (case_sensitiveness == CaseInsensitive) + { + auto key = Poco::toLower(name); + if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second) + throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique", + ErrorCodes::LOGICAL_ERROR); + case_insensitive_name_mapping[key] = name; + } } static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index 49c03049b92..5ef795c92d0 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -35,6 +35,8 @@ protected: return name; } + std::unordered_map case_insensitive_name_mapping; + public: /// For compatibility with SQL, it's possible to specify that certain function name is case insensitive. enum CaseSensitiveness @@ -68,9 +70,12 @@ public: factory_name + ": the alias name '" + alias_name + "' is already registered as real name", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive) + { if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second) throw Exception( factory_name + ": case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + case_insensitive_name_mapping[alias_name_lowercase] = real_name; + } if (!aliases.emplace(alias_name, real_dict_name).second) throw Exception(factory_name + ": alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); @@ -111,6 +116,15 @@ public: return getMap().count(name) || getCaseInsensitiveMap().count(name) || isAlias(name); } + /// Return the canonical name (the name used in registration) if it's different from `name`. + const String & getCanonicalNameIfAny(const String & name) const + { + auto it = case_insensitive_name_mapping.find(Poco::toLower(name)); + if (it != case_insensitive_name_mapping.end()) + return it->second; + return name; + } + virtual ~IFactoryWithAliases() override {} private: diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 768f1cfe487..09fd360a925 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -21,6 +21,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +const String & getFunctionCanonicalNameIfAny(const String & name) +{ + return FunctionFactory::instance().getCanonicalNameIfAny(name); +} void FunctionFactory::registerFunction(const std::string & name, @@ -36,10 +40,13 @@ void FunctionFactory::registerFunction(const throw Exception("FunctionFactory: the function name '" + name + "' is already registered as alias", ErrorCodes::LOGICAL_ERROR); - if (case_sensitiveness == CaseInsensitive - && !case_insensitive_functions.emplace(function_name_lowercase, creator).second) - throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique", - ErrorCodes::LOGICAL_ERROR); + if (case_sensitiveness == CaseInsensitive) + { + if (!case_insensitive_functions.emplace(function_name_lowercase, creator).second) + throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique", + ErrorCodes::LOGICAL_ERROR); + case_insensitive_name_mapping[function_name_lowercase] = name; + } } diff --git a/src/Functions/FunctionsRound.cpp b/src/Functions/FunctionsRound.cpp index b1349bd2164..c5ad27a0b90 100644 --- a/src/Functions/FunctionsRound.cpp +++ b/src/Functions/FunctionsRound.cpp @@ -8,7 +8,7 @@ namespace DB void registerFunctionsRound(FunctionFactory & factory) { factory.registerFunction("round", FunctionFactory::CaseInsensitive); - factory.registerFunction("roundBankers", FunctionFactory::CaseInsensitive); + factory.registerFunction("roundBankers", FunctionFactory::CaseSensitive); factory.registerFunction("floor", FunctionFactory::CaseInsensitive); factory.registerFunction("ceil", FunctionFactory::CaseInsensitive); factory.registerFunction("trunc", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/extractAllGroupsVertical.cpp b/src/Functions/extractAllGroupsVertical.cpp index 9cbd148b016..bf33eef70f3 100644 --- a/src/Functions/extractAllGroupsVertical.cpp +++ b/src/Functions/extractAllGroupsVertical.cpp @@ -18,7 +18,7 @@ namespace DB void registerFunctionExtractAllGroupsVertical(FunctionFactory & factory) { factory.registerFunction>(); - factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseInsensitive); + factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseSensitive); } } diff --git a/src/Interpreters/FunctionNameNormalizer.cpp b/src/Interpreters/FunctionNameNormalizer.cpp new file mode 100644 index 00000000000..f22f72b5e03 --- /dev/null +++ b/src/Interpreters/FunctionNameNormalizer.cpp @@ -0,0 +1,18 @@ +#include + +namespace DB +{ + +const String & getFunctionCanonicalNameIfAny(const String & name); +const String & getAggregateFunctionCanonicalNameIfAny(const String & name); + +void FunctionNameNormalizer::visit(ASTPtr & ast) +{ + if (auto * node_func = ast->as()) + node_func->name = getAggregateFunctionCanonicalNameIfAny(getFunctionCanonicalNameIfAny(node_func->name)); + + for (auto & child : ast->children) + visit(child); +} + +} diff --git a/src/Interpreters/FunctionNameNormalizer.h b/src/Interpreters/FunctionNameNormalizer.h new file mode 100644 index 00000000000..2b20c28bce0 --- /dev/null +++ b/src/Interpreters/FunctionNameNormalizer.h @@ -0,0 +1,14 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct FunctionNameNormalizer +{ + static void visit(ASTPtr &); +}; + +} diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 528b5ec6d8e..c393b214ee8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -442,10 +442,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) auto type_literal = std::make_shared(columns_desc.getPhysical(column).type->getName()); const auto & update_expr = kv.second; - auto updated_column = makeASTFunction("cast", + auto updated_column = makeASTFunction("CAST", makeASTFunction("if", getPartitionAndPredicateExpressionForMutationCommand(command), - makeASTFunction("cast", + makeASTFunction("CAST", update_expr->clone(), type_literal), std::make_shared(column)), diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index fd87d86bf97..cf4db8f174e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -934,6 +935,9 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & MarkTableIdentifiersVisitor::Data identifiers_data{aliases}; MarkTableIdentifiersVisitor(identifiers_data).visit(query); + /// Rewrite function names to their canonical ones. + FunctionNameNormalizer().visit(query); + /// Common subexpression elimination. Rewrite rules. QueryNormalizer::Data normalizer_data(aliases, settings); QueryNormalizer(normalizer_data).visit(query); diff --git a/src/Interpreters/addTypeConversionToAST.cpp b/src/Interpreters/addTypeConversionToAST.cpp index bb42ad79daa..18591fd732c 100644 --- a/src/Interpreters/addTypeConversionToAST.cpp +++ b/src/Interpreters/addTypeConversionToAST.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name) { - auto func = makeASTFunction("cast", ast, std::make_shared(type_name)); + auto func = makeASTFunction("CAST", ast, std::make_shared(type_name)); if (ASTWithAlias * ast_with_alias = dynamic_cast(ast.get())) { diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index eba03d7aa61..c9a96a81b48 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -43,7 +43,7 @@ void addDefaultRequiredExpressionsRecursively(const Block & block, const String RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); NameSet required_columns_names = columns_context.requiredColumns(); - auto cast_func = makeASTFunction("cast", column_default_expr, std::make_shared(columns.get(required_column).type->getName())); + auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared(columns.get(required_column).type->getName())); default_expr_list_accum->children.emplace_back(setAlias(cast_func, required_column)); added_columns.emplace(required_column); diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index d7a65c2f15d..1685688f02d 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -626,7 +626,7 @@ void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const I expr = makeASTFunction("assumeNotNull", std::move(expr)); } - expr = makeASTFunction("cast", std::move(expr), std::make_shared(result_column_type.getName())); + expr = makeASTFunction("CAST", std::move(expr), std::make_shared(result_column_type.getName())); if (null_as_default) { diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 9532d4b8ba2..7f7d59674bc 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout.decode() == 'database()\ndefault\n' + assert stdout.decode() == 'DATABASE()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default diff --git a/tests/queries/0_stateless/00597_push_down_predicate.reference b/tests/queries/0_stateless/00597_push_down_predicate.reference index 794d9e7af5f..bd1c4791df4 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -114,7 +114,7 @@ FROM ( SELECT 1 AS id, - identity(cast(1, \'UInt8\')) AS subquery + identity(CAST(1, \'UInt8\')) AS subquery WHERE subquery = 1 ) WHERE subquery = 1 diff --git a/tests/queries/0_stateless/01029_early_constant_folding.reference b/tests/queries/0_stateless/01029_early_constant_folding.reference index 8a1d4cec388..8a2d7e6c61a 100644 --- a/tests/queries/0_stateless/01029_early_constant_folding.reference +++ b/tests/queries/0_stateless/01029_early_constant_folding.reference @@ -2,7 +2,7 @@ SELECT 1 WHERE 0 SELECT 1 SELECT 1 -WHERE (1 IN (0, 2)) AND (2 = (identity(cast(2, \'UInt8\')) AS subquery)) +WHERE (1 IN (0, 2)) AND (2 = (identity(CAST(2, \'UInt8\')) AS subquery)) SELECT 1 WHERE 1 IN ( ( diff --git a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference index d10502c5860..e46fd479413 100644 --- a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference +++ b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference @@ -5,7 +5,7 @@ SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) FO 1,10 EXPLAIN SYNTAX SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n); SELECT - identity(cast(0, \'UInt64\')) AS n, + identity(CAST(0, \'UInt64\')) AS n, toUInt64(10 / n) SELECT * FROM (WITH (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) as q SELECT * FROM system.one WHERE q > 0); 0 diff --git a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference new file mode 100644 index 00000000000..5b0f7bdeb2d --- /dev/null +++ b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference @@ -0,0 +1,66 @@ +SELECT + CAST(1, 'INT'), + ceil(1), + ceil(1), + char(49), + CHAR_LENGTH('1'), + CHARACTER_LENGTH('1'), + coalesce(1), + concat('1', '1'), + corr(1, 1), + cos(1), + count(), + covarPop(1, 1), + covarSamp(1, 1), + DATABASE(), + dateDiff('DAY', toDate('2020-10-24'), toDate('2019-10-24')), + exp(1), + arrayFlatten([[1]]), + floor(1), + FQDN(), + greatest(1), + 1, + ifNull(1, 1), + lower('A'), + least(1), + length('1'), + log(1), + position('1', '1'), + log(1), + log10(1), + log2(1), + lower('A'), + max(1), + substring('123', 1, 1), + min(1), + 1 % 1, + NOT 1, + now(), + now64(), + nullIf(1, 1), + pi(), + position('123', '2'), + pow(1, 1), + pow(1, 1), + rand(), + replaceAll('1', '1', '2'), + reverse('123'), + round(1), + sin(1), + sqrt(1), + stddevPop(1), + stddevSamp(1), + substring('123', 2), + substring('123', 2), + count(), + tan(1), + tanh(1), + trunc(1), + trunc(1), + upper('A'), + upper('A'), + currentUser(), + varPop(1), + varSamp(1), + toWeek(toDate('2020-10-24')), + toYearWeek(toDate('2020-10-24')) diff --git a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql new file mode 100644 index 00000000000..9b35087182c --- /dev/null +++ b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql @@ -0,0 +1 @@ +EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH('1'), CHARACTER_LENGTH('1'), COALESCE(1), CONCAT('1', '1'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), DATEDIFF('DAY', toDate('2020-10-24'), toDate('2019-10-24')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE('A'), LEAST(1), LENGTH('1'), LN(1), LOCATE('1', '1'), LOG(1), LOG10(1), LOG2(1), LOWER('A'), MAX(1), MID('123', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION('123', '2'), POW(1, 1), POWER(1, 1), RAND(), REPLACE('1', '1', '2'), REVERSE('123'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR('123', 2), SUBSTRING('123', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE('A'), UPPER('A'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate('2020-10-24')), YEARWEEK(toDate('2020-10-24')) format TSVRaw; From 2dc7ba160a3bdc61765b12336edf753a0100f923 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 14 Feb 2021 20:53:50 +0800 Subject: [PATCH 02/20] Better --- src/Interpreters/FunctionNameNormalizer.cpp | 27 +++++++++++++++++-- src/Interpreters/FunctionNameNormalizer.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 ++ ...OptimizeIfWithConstantConditionVisitor.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- ...56_test_query_log_factories_info.reference | 2 +- 8 files changed, 33 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/FunctionNameNormalizer.cpp b/src/Interpreters/FunctionNameNormalizer.cpp index f22f72b5e03..36ccc9340ea 100644 --- a/src/Interpreters/FunctionNameNormalizer.cpp +++ b/src/Interpreters/FunctionNameNormalizer.cpp @@ -1,18 +1,41 @@ #include +#include +#include + namespace DB { const String & getFunctionCanonicalNameIfAny(const String & name); const String & getAggregateFunctionCanonicalNameIfAny(const String & name); -void FunctionNameNormalizer::visit(ASTPtr & ast) +void FunctionNameNormalizer::visit(IAST * ast) { + if (!ast) + return; + + if (auto * node_storage = ast->as()) + { + visit(node_storage->partition_by); + visit(node_storage->primary_key); + visit(node_storage->order_by); + visit(node_storage->sample_by); + visit(node_storage->ttl_table); + return; + } + + if (auto * node_decl = ast->as()) + { + visit(node_decl->default_expression.get()); + visit(node_decl->ttl.get()); + return; + } + if (auto * node_func = ast->as()) node_func->name = getAggregateFunctionCanonicalNameIfAny(getFunctionCanonicalNameIfAny(node_func->name)); for (auto & child : ast->children) - visit(child); + visit(child.get()); } } diff --git a/src/Interpreters/FunctionNameNormalizer.h b/src/Interpreters/FunctionNameNormalizer.h index 2b20c28bce0..3f22bb2f627 100644 --- a/src/Interpreters/FunctionNameNormalizer.h +++ b/src/Interpreters/FunctionNameNormalizer.h @@ -8,7 +8,7 @@ namespace DB struct FunctionNameNormalizer { - static void visit(ASTPtr &); + static void visit(IAST *); }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e9a11b9eb0d..bc38d4e3821 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -56,6 +56,7 @@ #include #include #include +#include #include #include @@ -1118,6 +1119,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons BlockIO InterpreterCreateQuery::execute() { + FunctionNameNormalizer().visit(query_ptr.get()); auto & create = query_ptr->as(); if (!create.cluster.empty()) { diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index dee4c69118b..cdcf6f7dddd 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -29,7 +29,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v /// cast of numeric constant in condition to UInt8 if (const auto * function = condition->as()) { - if (function->name == "cast") + if (function->name == "CAST") { if (const auto * expr_list = function->arguments->as()) { diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cf4db8f174e..7b1a960d435 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -936,7 +936,7 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & MarkTableIdentifiersVisitor(identifiers_data).visit(query); /// Rewrite function names to their canonical ones. - FunctionNameNormalizer().visit(query); + FunctionNameNormalizer().visit(query.get()); /// Common subexpression elimination. Rewrite rules. QueryNormalizer::Data normalizer_data(aliases, settings); diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index c9a96a81b48..d06cde99425 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -79,7 +79,7 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi continue; auto cast_func = makeASTFunction( - "cast", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); + "CAST", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); conversion_expr_list->children.emplace_back(setAlias(cast_func, required_column.name)); diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3d868812304..7a426e7774d 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -864,7 +864,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect expr_list_args->children.push_back(std::move(type_literal)); auto func_node = std::make_shared(); - func_node->name = "cast"; + func_node->name = "CAST"; func_node->arguments = std::move(expr_list_args); func_node->children.push_back(func_node->arguments); diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index 3c93cd9ec26..324890c0a5a 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -11,7 +11,7 @@ arraySort(used_table_functions) ['numbers'] arraySort(used_functions) -['addDays','array','arrayFlatten','cast','modulo','plus','substring','toDate','toDayOfYear','toTypeName','toWeek'] +['CAST','addDays','array','arrayFlatten','modulo','plus','substring','toDate','toDayOfYear','toTypeName','toWeek'] arraySort(used_data_type_families) ['Array','Int32','Nullable','String'] From cac9c7fc079835b4e26cf2b5ff8ad776b1369c5d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 15 Feb 2021 00:00:47 +0800 Subject: [PATCH 03/20] Fix tests --- tests/queries/0_stateless/00642_cast.reference | 4 ++-- tests/queries/0_stateless/00643_cast_zookeeper.reference | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00642_cast.reference b/tests/queries/0_stateless/00642_cast.reference index 3d5572932fb..7f5333f590e 100644 --- a/tests/queries/0_stateless/00642_cast.reference +++ b/tests/queries/0_stateless/00642_cast.reference @@ -10,11 +10,11 @@ hello CREATE TABLE default.cast ( `x` UInt8, - `e` Enum8('hello' = 1, 'world' = 2) DEFAULT cast(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') + `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') ) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192 x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT cast(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper.reference index 658233be742..9123463de1a 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -1,12 +1,12 @@ CREATE TABLE default.cast1 ( `x` UInt8, - `e` Enum8('hello' = 1, 'world' = 2) DEFAULT cast(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') + `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192 x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT cast(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello 1 hello From f402aa4057814078b7b7ef2e0175ab2753d2bced Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 17 Feb 2021 23:36:37 +0800 Subject: [PATCH 04/20] Normalize constant expression --- src/Interpreters/evaluateConstantExpression.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 02ef3426483..70b9baa544f 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -35,6 +36,7 @@ std::pair> evaluateConstantExpression(co auto ast = node->clone(); ReplaceQueryParameterVisitor param_visitor(context.getQueryParameters()); param_visitor.visit(ast); + FunctionNameNormalizer().visit(ast.get()); String name = ast->getColumnName(); auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); From 2c4bc43014c510292340954647fbebf0f72620e9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 18 Feb 2021 11:27:24 +0800 Subject: [PATCH 05/20] Backward compatible --- src/Core/Settings.h | 1 + src/Interpreters/TreeRewriter.cpp | 3 ++- src/Interpreters/evaluateConstantExpression.cpp | 5 ++++- src/Server/TCPHandler.cpp | 6 ++++++ 4 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9bb9ad30f15..4c5fe93bb03 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -383,6 +383,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ + M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 7b1a960d435..37f49874e0a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -936,7 +936,8 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & MarkTableIdentifiersVisitor(identifiers_data).visit(query); /// Rewrite function names to their canonical ones. - FunctionNameNormalizer().visit(query.get()); + if (settings.normalize_function_names) + FunctionNameNormalizer().visit(query.get()); /// Common subexpression elimination. Rewrite rules. QueryNormalizer::Data normalizer_data(aliases, settings); diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 70b9baa544f..42e96bae07b 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -36,7 +36,10 @@ std::pair> evaluateConstantExpression(co auto ast = node->clone(); ReplaceQueryParameterVisitor param_visitor(context.getQueryParameters()); param_visitor.visit(ast); - FunctionNameNormalizer().visit(ast.get()); + + if (context.getSettingsRef().normalize_function_names) + FunctionNameNormalizer().visit(ast.get()); + String name = ast->getColumnName(); auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c207d188a85..430a01bb97a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1133,6 +1133,12 @@ void TCPHandler::receiveQuery() } query_context->applySettingsChanges(settings_changes); + /// Disable function name normalization it's not an initial query. + if (client_info.query_kind != ClientInfo::QueryKind::INITIAL_QUERY) + { + query_context->setSetting("normalize_function_names", Field(0)); + } + // Use the received query id, or generate a random default. It is convenient // to also generate the default OpenTelemetry trace id at the same time, and // set the trace parent. From fc185e5fb73dc0ac82ab8b0b7a79518832401379 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 19 Feb 2021 11:56:24 +0800 Subject: [PATCH 06/20] Another try --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 430a01bb97a..9794a86d3e3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1133,8 +1133,8 @@ void TCPHandler::receiveQuery() } query_context->applySettingsChanges(settings_changes); - /// Disable function name normalization it's not an initial query. - if (client_info.query_kind != ClientInfo::QueryKind::INITIAL_QUERY) + /// Disable function name normalization it's a secondary query. + if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { query_context->setSetting("normalize_function_names", Field(0)); } From fc1885ea9b01714290fba8ee8fbbe1a78894e573 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 19 Feb 2021 17:28:01 +0800 Subject: [PATCH 07/20] Try fixing flaky tests --- tests/queries/0_stateless/00643_cast_zookeeper.sql | 2 ++ .../queries/0_stateless/01656_test_query_log_factories_info.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper.sql index c52d44bd88b..c9760f00ca7 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.sql +++ b/tests/queries/0_stateless/00643_cast_zookeeper.sql @@ -1,3 +1,5 @@ +SET database_atomic_wait_for_drop_and_detach_synchronously=1; + DROP TABLE IF EXISTS cast1; DROP TABLE IF EXISTS cast2; diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql index 9f374def8b5..17657cf60f5 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql @@ -1,3 +1,5 @@ +SET database_atomic_wait_for_drop_and_detach_synchronously=1; + SELECT uniqArray([1, 1, 2]), SUBSTRING('Hello, world', 7, 5), flatten([[[BIT_AND(123)]], [[mod(3, 2)], [CAST('1' AS INTEGER)]]]), From 7c04f15c8031a63f20573b9948dd18005f860f26 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 09:11:42 +0300 Subject: [PATCH 08/20] Add log message when stacktrace cannot be obtained for thread This is to provide better diagnostics for 01051_system_stack_trace failure [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/20881/866dfaec793f764dc9ba167d3ac9f6521b9b3381/functional_stateless_tests_(release,_wide_parts_enabled).html#fail1 --- src/Storages/System/StorageSystemStackTrace.cpp | 4 ++++ src/Storages/System/StorageSystemStackTrace.h | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index abb2fdf54ed..e74d56108ad 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -150,6 +151,7 @@ namespace StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) : IStorageSystemOneBlock(table_id_) + , log(&Poco::Logger::get("StorageSystemStackTrace")) { notification_pipe.open(); @@ -229,6 +231,8 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte } else { + LOG_DEBUG(log, "Cannot obtain a stack trace for thread {}", tid); + /// Cannot obtain a stack trace. But create a record in result nevertheless. res_columns[0]->insert(tid); diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index a389f02eb09..582618d2ecd 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -6,6 +6,10 @@ #include #include +namespace Poco +{ +class Logger; +} namespace DB { @@ -30,6 +34,8 @@ protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; mutable std::mutex mutex; + + Poco::Logger * log; }; } From 2ab37d025a62f650d4b90f5fafa23f4076ab3844 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 20 Feb 2021 16:14:38 +0800 Subject: [PATCH 09/20] Skip non-parallel tests --- tests/queries/skip_list.json | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index fdb845b7e72..1164d7b0004 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -364,6 +364,7 @@ "00626_replace_partition_from_table", "00626_replace_partition_from_table_zookeeper", "00633_materialized_view_and_too_many_parts_zookeeper", + "00643_cast_zookeeper", "00652_mergetree_mutations", "00652_replicated_mutations_zookeeper", "00682_empty_parts_merge", @@ -577,10 +578,11 @@ "01602_show_create_view", "01603_rename_overwrite_bug", "01646_system_restart_replicas_smoke", // system restart replicas is a global query - "01676_dictget_in_default_expression", - "01715_background_checker_blather_zookeeper", - "01700_system_zookeeper_path_in", + "01656_test_query_log_factories_info", "01669_columns_declaration_serde", + "01676_dictget_in_default_expression", + "01700_system_zookeeper_path_in", + "01715_background_checker_blather_zookeeper", "attach", "ddl_dictionaries", "dictionary", From f37631830f8139a68c42111c11584956f992630a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 20 Feb 2021 16:45:25 +0800 Subject: [PATCH 10/20] Comments --- src/Interpreters/FunctionNameNormalizer.cpp | 4 ++++ src/Interpreters/ya.make | 1 + src/Server/TCPHandler.cpp | 4 +++- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/FunctionNameNormalizer.cpp b/src/Interpreters/FunctionNameNormalizer.cpp index 36ccc9340ea..255f4d8c6bb 100644 --- a/src/Interpreters/FunctionNameNormalizer.cpp +++ b/src/Interpreters/FunctionNameNormalizer.cpp @@ -14,6 +14,8 @@ void FunctionNameNormalizer::visit(IAST * ast) if (!ast) return; + // Normalize only selected children. Avoid normalizing engine clause because some engine might + // have the same name as function, e.g. Log. if (auto * node_storage = ast->as()) { visit(node_storage->partition_by); @@ -24,6 +26,8 @@ void FunctionNameNormalizer::visit(IAST * ast) return; } + // Normalize only selected children. Avoid normalizing type clause because some type might + // have the same name as function, e.g. Date. if (auto * node_decl = ast->as()) { visit(node_decl->default_expression.get()); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index cd4980927e4..e7882ec8d98 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -58,6 +58,7 @@ SRCS( ExternalModelsLoader.cpp ExtractExpressionInfoVisitor.cpp FillingRow.cpp + FunctionNameNormalizer.cpp HashJoin.cpp IExternalLoadable.cpp IInterpreter.cpp diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9794a86d3e3..d2ce2a409a9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1133,7 +1133,9 @@ void TCPHandler::receiveQuery() } query_context->applySettingsChanges(settings_changes); - /// Disable function name normalization it's a secondary query. + /// Disable function name normalization when it's a secondary query, because queries are either + /// already normalized on initiator node, or not normalized and should remain unnormalized for + /// compatibility. if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { query_context->setSetting("normalize_function_names", Field(0)); From 6873ad19839e61f0233de80dab7bd52da6fd0254 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 21:04:19 +0300 Subject: [PATCH 11/20] tests/integration: fix yamllint issues in docker-compose.yaml for mysql --- .../docker_compose_mysql_5_7_for_materialize_mysql.yml | 7 +++++-- .../docker_compose_mysql_8_0_for_materialize_mysql.yml | 8 ++++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml index e7d762203ee..f238c1aad56 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml @@ -6,5 +6,8 @@ services: environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 3308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + - 3308:3306 + command: --server_id=100 --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml index 918a2b5f80f..4e4648ee960 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml @@ -6,5 +6,9 @@ services: environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 33308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + - 33308:3306 + command: --server_id=100 --log-bin='mysql-bin-1.log' + --default_authentication_plugin='mysql_native_password' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency From 80f448a466c2d450e253443004f905c55c8b9907 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 21:04:19 +0300 Subject: [PATCH 12/20] tests/integration: preserve mysql docker logs in test_materialize_mysql_database --- tests/integration/helpers/cluster.py | 4 ++-- .../test_materialize_mysql_database/test.py | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index aaba3a34555..16ceb823f2e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -44,8 +44,8 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path -def run_and_check(args, env=None, shell=False): - res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) +def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE): + res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index e55772d9e1d..58b20ad3139 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -37,6 +37,12 @@ class MySQLNodeInstance: self.docker_compose = docker_compose self.project_name = project_name + self.base_dir = p.dirname(__file__) + self.instances_dir = p.join(self.base_dir, '_instances_mysql') + if not os.path.exists(self.instances_dir): + os.mkdir(self.instances_dir) + self.docker_logs_path = p.join(self.instances_dir, 'docker_mysql.log') + def alloc_connection(self): if self.mysql_connection is None: @@ -75,6 +81,16 @@ class MySQLNodeInstance: if self.mysql_connection is not None: self.mysql_connection.close() + with open(self.docker_logs_path, "w+") as f: + try: + run_and_check([ + 'docker-compose', + '-p', cluster.project_name, + '-f', self.docker_compose, 'logs', + ], stdout=f) + except Exception as e: + print("Unable to get logs from docker mysql.") + def wait_mysql_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: From d343060dd5ccd521329af546d4157775ab01bfb6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 21:04:19 +0300 Subject: [PATCH 13/20] tests/integration: add start_and_wait() in test_materialize_mysql_database --- .../test_materialize_mysql_database/test.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 58b20ad3139..ced9a978d02 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -77,6 +77,14 @@ class MySQLNodeInstance: cursor.execute(executio_query) return cursor.fetchall() + def start_and_wait(self): + run_and_check(['docker-compose', + '-p', cluster.project_name, + '-f', self.docker_compose, + 'up', '--no-recreate', '-d', + ]) + self.wait_mysql_to_start(120) + def close(self): if self.mysql_connection is not None: self.mysql_connection.close() @@ -111,9 +119,7 @@ def started_mysql_5_7(): mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose) try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) + mysql_node.start_and_wait() yield mysql_node finally: mysql_node.close() @@ -127,9 +133,7 @@ def started_mysql_8_0(): mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose) try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) + mysql_node.start_and_wait() yield mysql_node finally: mysql_node.close() From 4784d09d1701c44781c3441efed5506f0d14a9dc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 21:04:20 +0300 Subject: [PATCH 14/20] tests/integration: increase verbosity for test_materialize_mysql_database --- .../compose/docker_compose_mysql_5_7_for_materialize_mysql.yml | 1 + .../compose/docker_compose_mysql_8_0_for_materialize_mysql.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml index f238c1aad56..5aa13ba91c7 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml @@ -11,3 +11,4 @@ services: --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + --log-error-verbosity=3 diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml index 4e4648ee960..7c8a930c84e 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml @@ -12,3 +12,4 @@ services: --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + --log-error-verbosity=3 From a33183ee1e12e1d4072054f0908732955683a0e6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 14 Feb 2021 16:07:20 +0000 Subject: [PATCH 15/20] fix transform with floating point key better update test --- src/Functions/transform.cpp | 102 +++++++++++------- .../01704_transform_with_float_key.reference | 30 ++++++ .../01704_transform_with_float_key.sql | 3 + 3 files changed, 94 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/01704_transform_with_float_key.reference create mode 100644 tests/queries/0_stateless/01704_transform_with_float_key.sql diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index ab589be9c07..07fbd5a7b96 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -1,4 +1,6 @@ #include +#include + #include #include #include @@ -13,6 +15,7 @@ #include #include #include +#include namespace DB @@ -491,7 +494,7 @@ private: dst.resize(size); for (size_t i = 0; i < size; ++i) { - auto it = table.find(src[i]); + const auto * it = table.find(ext::bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian. else @@ -507,7 +510,7 @@ private: dst.resize(size); for (size_t i = 0; i < size; ++i) { - auto it = table.find(src[i]); + const auto * it = table.find(ext::bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian. else @@ -523,7 +526,7 @@ private: dst.resize(size); for (size_t i = 0; i < size; ++i) { - auto it = table.find(src[i]); + const auto * it = table.find(ext::bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); else @@ -541,7 +544,7 @@ private: ColumnString::Offset current_dst_offset = 0; for (size_t i = 0; i < size; ++i) { - auto it = table.find(src[i]); + const auto * it = table.find(ext::bit_cast(src[i])); StringRef ref = it ? it->getMapped() : dst_default; dst_data.resize(current_dst_offset + ref.size); memcpy(&dst_data[current_dst_offset], ref.data, ref.size); @@ -562,7 +565,8 @@ private: ColumnString::Offset current_dst_default_offset = 0; for (size_t i = 0; i < size; ++i) { - auto it = table.find(src[i]); + Field key = src[i]; + const auto * it = table.find(key.reinterpret()); StringRef ref; if (it) @@ -778,50 +782,66 @@ private: /// Note: Doesn't check the duplicates in the `from` array. - if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String) + const IDataType & from_type = *arguments[0].type; + + if (from[0].getType() != Field::Types::String) { - cache.table_num_to_num = std::make_unique(); - auto & table = *cache.table_num_to_num; - for (size_t i = 0; i < size; ++i) + if (to[0].getType() != Field::Types::String) { - // Field may be of Float type, but for the purpose of bitwise - // equality we can treat them as UInt64, hence the reinterpret(). - table[from[i].reinterpret()] = (*used_to)[i].reinterpret(); + cache.table_num_to_num = std::make_unique(); + auto & table = *cache.table_num_to_num; + for (size_t i = 0; i < size; ++i) + { + Field key = convertFieldToType(from[i], from_type); + if (key.isNull()) + continue; + + // Field may be of Float type, but for the purpose of bitwise + // equality we can treat them as UInt64, hence the reinterpret(). + table[key.reinterpret()] = (*used_to)[i].reinterpret(); + } + } + else + { + cache.table_num_to_string = std::make_unique(); + auto & table = *cache.table_num_to_string; + for (size_t i = 0; i < size; ++i) + { + Field key = convertFieldToType(from[i], from_type); + if (key.isNull()) + continue; + + const String & str_to = to[i].get(); + StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; + table[key.reinterpret()] = ref; + } } } - else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String) + else { - cache.table_num_to_string = std::make_unique(); - auto & table = *cache.table_num_to_string; - for (size_t i = 0; i < size; ++i) + if (to[0].getType() != Field::Types::String) { - const String & str_to = to[i].get(); - StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; - table[from[i].reinterpret()] = ref; + cache.table_string_to_num = std::make_unique(); + auto & table = *cache.table_string_to_num; + for (size_t i = 0; i < size; ++i) + { + const String & str_from = from[i].get(); + StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; + table[ref] = (*used_to)[i].reinterpret(); + } } - } - else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String) - { - cache.table_string_to_num = std::make_unique(); - auto & table = *cache.table_string_to_num; - for (size_t i = 0; i < size; ++i) + else { - const String & str_from = from[i].get(); - StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; - table[ref] = (*used_to)[i].reinterpret(); - } - } - else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String) - { - cache.table_string_to_string = std::make_unique(); - auto & table = *cache.table_string_to_string; - for (size_t i = 0; i < size; ++i) - { - const String & str_from = from[i].get(); - const String & str_to = to[i].get(); - StringRef ref_from{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; - StringRef ref_to{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; - table[ref_from] = ref_to; + cache.table_string_to_string = std::make_unique(); + auto & table = *cache.table_string_to_string; + for (size_t i = 0; i < size; ++i) + { + const String & str_from = from[i].get(); + const String & str_to = to[i].get(); + StringRef ref_from{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; + StringRef ref_to{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; + table[ref_from] = ref_to; + } } } diff --git a/tests/queries/0_stateless/01704_transform_with_float_key.reference b/tests/queries/0_stateless/01704_transform_with_float_key.reference new file mode 100644 index 00000000000..761e15c903c --- /dev/null +++ b/tests/queries/0_stateless/01704_transform_with_float_key.reference @@ -0,0 +1,30 @@ +- +Hello +- +World +- +- +- +- +- +- +- +- +Hello +- +World +- +- +- +- +- +- +- +Hello +- +World +- +- +- +- +- diff --git a/tests/queries/0_stateless/01704_transform_with_float_key.sql b/tests/queries/0_stateless/01704_transform_with_float_key.sql new file mode 100644 index 00000000000..690c73ee28a --- /dev/null +++ b/tests/queries/0_stateless/01704_transform_with_float_key.sql @@ -0,0 +1,3 @@ +SELECT transform(number / 2, [0.5, 1.5], ['Hello', 'World'], '-') FROM numbers(10); +SELECT transform(number / 2, [1.0, 2.0], ['Hello', 'World'], '-') FROM numbers(10); +SELECT transform(number / 2, [1, 2], ['Hello', 'World'], '-') FROM numbers(10); From 9a264091473bc303fdff39e90d67cb9c52c24b3d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 Feb 2021 10:56:48 +0300 Subject: [PATCH 16/20] Fix performance tests (by avoid sharding status file for right and left server) Since cp -al (hard links): 2021.02.21 01:09:09.991771 [ 243 ] {} StatusFile: Status file right/db/status already exists - unclean restart. Contents: PID: 241 Started at: 2021-02-21 01:09:09 Revision: 54448 2021.02.21 01:09:09.992007 [ 243 ] {} Application: DB::Exception: Cannot lock file right/db/status. Another server instance in same directory is already running. --- docker/test/performance-comparison/compare.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 9a0d8093a55..2b19a5e75a8 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -97,6 +97,7 @@ function configure rm -r right/db ||: rm -r db0/preprocessed_configs ||: rm -r db0/{data,metadata}/system ||: + rm db0/status ||: cp -al db0/ left/db/ cp -al db0/ right/db/ } From 9cb972e9ed740de4f33f55da96518106b121332a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Feb 2021 13:23:18 +0300 Subject: [PATCH 17/20] Fix UBSan report in Decimal arithmetic #19432 --- base/common/arithmeticOverflow.h | 50 ++++++++++++------- .../0_stateless/01732_bigint_ubsan.reference | 0 .../0_stateless/01732_bigint_ubsan.sql | 11 ++++ 3 files changed, 42 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/01732_bigint_ubsan.reference create mode 100644 tests/queries/0_stateless/01732_bigint_ubsan.sql diff --git a/base/common/arithmeticOverflow.h b/base/common/arithmeticOverflow.h index fd557fd5b2d..a92fe56b9cb 100644 --- a/base/common/arithmeticOverflow.h +++ b/base/common/arithmeticOverflow.h @@ -6,6 +6,25 @@ namespace common { + /// Multiply and ignore overflow. + template + inline auto NO_SANITIZE_UNDEFINED mulIgnoreOverflow(T1 x, T2 y) + { + return x * y; + } + + template + inline auto NO_SANITIZE_UNDEFINED addIgnoreOverflow(T1 x, T2 y) + { + return x + y; + } + + template + inline auto NO_SANITIZE_UNDEFINED subIgnoreOverflow(T1 x, T2 y) + { + return x - y; + } + template inline bool addOverflow(T x, T y, T & res) { @@ -35,14 +54,14 @@ namespace common { static constexpr __int128 min_int128 = minInt128(); static constexpr __int128 max_int128 = maxInt128(); - res = x + y; + res = addIgnoreOverflow(x, y); return (y > 0 && x > max_int128 - y) || (y < 0 && x < min_int128 - y); } template <> inline bool addOverflow(wInt256 x, wInt256 y, wInt256 & res) { - res = x + y; + res = addIgnoreOverflow(x, y); return (y > 0 && x > std::numeric_limits::max() - y) || (y < 0 && x < std::numeric_limits::min() - y); } @@ -50,7 +69,7 @@ namespace common template <> inline bool addOverflow(wUInt256 x, wUInt256 y, wUInt256 & res) { - res = x + y; + res = addIgnoreOverflow(x, y); return x > std::numeric_limits::max() - y; } @@ -83,14 +102,14 @@ namespace common { static constexpr __int128 min_int128 = minInt128(); static constexpr __int128 max_int128 = maxInt128(); - res = x - y; + res = subIgnoreOverflow(x, y); return (y < 0 && x > max_int128 + y) || (y > 0 && x < min_int128 + y); } template <> inline bool subOverflow(wInt256 x, wInt256 y, wInt256 & res) { - res = x - y; + res = subIgnoreOverflow(x, y); return (y < 0 && x > std::numeric_limits::max() + y) || (y > 0 && x < std::numeric_limits::min() + y); } @@ -98,7 +117,7 @@ namespace common template <> inline bool subOverflow(wUInt256 x, wUInt256 y, wUInt256 & res) { - res = x - y; + res = subIgnoreOverflow(x, y); return x < y; } @@ -129,40 +148,33 @@ namespace common template <> inline bool mulOverflow(__int128 x, __int128 y, __int128 & res) { - res = static_cast(x) * static_cast(y); /// Avoid signed integer overflow. + res = mulIgnoreOverflow(x, y); if (!x || !y) return false; unsigned __int128 a = (x > 0) ? x : -x; unsigned __int128 b = (y > 0) ? y : -y; - return (a * b) / b != a; + return mulIgnoreOverflow(a, b) / b != a; } template <> inline bool mulOverflow(wInt256 x, wInt256 y, wInt256 & res) { - res = x * y; + res = mulIgnoreOverflow(x, y); if (!x || !y) return false; wInt256 a = (x > 0) ? x : -x; wInt256 b = (y > 0) ? y : -y; - return (a * b) / b != a; + return mulIgnoreOverflow(a, b) / b != a; } template <> inline bool mulOverflow(wUInt256 x, wUInt256 y, wUInt256 & res) { - res = x * y; + res = mulIgnoreOverflow(x, y); if (!x || !y) return false; - return (x * y) / y != x; - } - - /// Multiply and ignore overflow. - template - inline auto NO_SANITIZE_UNDEFINED mulIgnoreOverflow(T1 x, T2 y) - { - return x * y; + return res / y != x; } } diff --git a/tests/queries/0_stateless/01732_bigint_ubsan.reference b/tests/queries/0_stateless/01732_bigint_ubsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01732_bigint_ubsan.sql b/tests/queries/0_stateless/01732_bigint_ubsan.sql new file mode 100644 index 00000000000..238a5d99d30 --- /dev/null +++ b/tests/queries/0_stateless/01732_bigint_ubsan.sql @@ -0,0 +1,11 @@ +CREATE TEMPORARY TABLE decimal +( + f dec(38, 38) +); + +INSERT INTO decimal VALUES (0); +INSERT INTO decimal VALUES (0.42); +INSERT INTO decimal VALUES (-0.42); + +SELECT f + 1048575, f - 21, f - 84, f * 21, f * -21, f / 21, f / 84 FROM decimal WHERE f > 0; -- { serverError 407 } +SELECT f + -2, f - 21, f - 84, f * 21, f * -21, f / 9223372036854775807, f / 84 FROM decimal WHERE f > 0; -- { serverError 407 } From 4085782b05657f8f7ad670dd563ca2d75a63bce5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Feb 2021 15:34:48 +0300 Subject: [PATCH 18/20] More gentle exception messages --- src/Functions/FunctionsConversion.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b95d4ea9790..74d8d853dcb 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1313,7 +1313,7 @@ public: else if constexpr (std::is_same_v) return createDecimalMaxPrecision(scale); - throw Exception("Something wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR); + throw Exception("Unexpected branch in code of conversion function: it is a bug.", ErrorCodes::LOGICAL_ERROR); } else { @@ -1337,7 +1337,7 @@ public: if constexpr (std::is_same_v) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); else if constexpr (std::is_same_v) - throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Unexpected branch in code of conversion function: it is a bug.", ErrorCodes::LOGICAL_ERROR); else return std::make_shared(); } From 75edfd0549e80223371a87807b90a7006f02b7f3 Mon Sep 17 00:00:00 2001 From: Michael Monashev Date: Sun, 21 Feb 2021 15:56:47 +0300 Subject: [PATCH 19/20] Update lowcardinality.md --- docs/ru/sql-reference/data-types/lowcardinality.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md index d94cedd29ce..1e83cdbc58c 100644 --- a/docs/ru/sql-reference/data-types/lowcardinality.md +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -23,7 +23,7 @@ LowCardinality(data_type) Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. -При работе со строками, использование `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. +При работе со строками, используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. ## Пример From f93b450da9ea6f80f7cb50e6bf374eda2d73498a Mon Sep 17 00:00:00 2001 From: Michael Monashev Date: Sun, 21 Feb 2021 16:00:52 +0300 Subject: [PATCH 20/20] Update lowcardinality.md --- docs/ru/sql-reference/data-types/lowcardinality.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md index 1e83cdbc58c..3b5b337d731 100644 --- a/docs/ru/sql-reference/data-types/lowcardinality.md +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -23,7 +23,7 @@ LowCardinality(data_type) Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. -При работе со строками, используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. +При работе со строками используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. ## Пример