From 75117389eccf862b1a08b93a32d4f839846715f6 Mon Sep 17 00:00:00 2001 From: M0r64n Date: Tue, 16 Feb 2021 18:50:11 +0400 Subject: [PATCH 01/48] Add a couple of QOL file engine settings --- docs/en/operations/settings/settings.md | 20 +++++++++++++++++++ src/Core/Settings.h | 2 ++ src/Storages/StorageFile.cpp | 12 ++++++++++- ..._engine_file_empty_if_not_exists.reference | 0 .../01720_engine_file_empty_if_not_exists.sql | 15 ++++++++++++++ ...1_engine_file_truncate_on_insert.reference | 13 ++++++++++++ .../01721_engine_file_truncate_on_insert.sql | 20 +++++++++++++++++++ 7 files changed, 81 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.reference create mode 100644 tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql create mode 100644 tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference create mode 100644 tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 43519bfc8dc..6440f09bb40 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2659,3 +2659,23 @@ Result: Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md) behaviour. [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) + +## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists} + +Allows to select data from a file engine table without file. + +Possible values: +- 0 — `SELECT` throws exception. +- 1 — `SELECT` returns empty result. + +Default value: `0`. + +## engine_file_truncate_on_insert {#engine-file-truncate-on-insert} + +Enables or disables truncate before insert in file engine tables. + +Possible values: +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9bb9ad30f15..98c3b9d1f85 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -421,6 +421,8 @@ class IColumn; M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \ M(Bool, allow_experimental_query_deduplication, false, "Allow sending parts' UUIDs for a query in order to deduplicate data parts if any", 0) \ + M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ + M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a5935ba3bf4..856d03ea2ce 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; @@ -427,7 +428,12 @@ Pipe StorageFile::read( paths = {""}; /// when use fd, paths are empty else if (paths.size() == 1 && !Poco::File(paths[0]).exists()) - throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST); + { + if (context.getSettingsRef().engine_file_empty_if_not_exists) + return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); + else + throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST); + } auto files_info = std::make_shared(); @@ -547,6 +553,10 @@ BlockOutputStreamPtr StorageFile::write( throw Exception("Method write is not implemented for Distributed format", ErrorCodes::NOT_IMPLEMENTED); std::string path; + if (context.getSettingsRef().engine_file_truncate_on_insert) + if (0 != ::truncate(paths[0].c_str(), 0)) + throwFromErrnoWithPath("Cannot truncate file " + paths[0], paths[0], ErrorCodes::CANNOT_TRUNCATE_FILE); + if (!paths.empty()) { path = paths[0]; diff --git a/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.reference b/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql b/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql new file mode 100644 index 00000000000..c04e01ccc88 --- /dev/null +++ b/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS file_engine_table; + +CREATE TABLE file_engine_table (id UInt32) ENGINE=File(TSV); + +SELECT * FROM file_engine_table; --{ serverError 107 } + +SET engine_file_empty_if_not_exists=0; + +SELECT * FROM file_engine_table; --{ serverError 107 } + +SET engine_file_empty_if_not_exists=1; + +SELECT * FROM file_engine_table; + +SET engine_file_empty_if_not_exists=0; diff --git a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference new file mode 100644 index 00000000000..a25fb4f0e7e --- /dev/null +++ b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference @@ -0,0 +1,13 @@ +1 +2 +3 +4 +1 +2 +3 +4 +5 +6 +0 +1 +2 \ No newline at end of file diff --git a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql new file mode 100644 index 00000000000..65246db7963 --- /dev/null +++ b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql @@ -0,0 +1,20 @@ +INSERT INTO TABLE FUNCTION file('01718_file/test/data.TSV', 'TSV', 'id UInt32') VALUES ('file', 42); +ATTACH TABLE test FROM '01718_file/test' (id UInt8) ENGINE=File(TSV); + +CREATE TABLE file_engine_table (id UInt32) ENGINE=File(TabSeparated); + +INSERT INTO file_engine_table VALUES (1), (2), (3); +INSERT INTO file_engine_table VALUES (4); +SELECT * FROM file_engine_table; + +SET engine_file_truncate_on_insert=0; + +INSERT INTO file_engine_table VALUES (5), (6); +SELECT * FROM file_engine_table; + +SET engine_file_truncate_on_insert=1; + +INSERT INTO file_engine_table VALUES (0), (1), (2); +SELECT * FROM file_engine_table; + +SET engine_file_truncate_on_insert=0; From fa200160915ee9c187e5e64a4a1e395d70430b7f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Feb 2021 09:53:18 +0300 Subject: [PATCH 02/48] Enable distributed_aggregation_memory_efficient by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9bb9ad30f15..6c05d247037 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -100,7 +100,7 @@ class IColumn; M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ M(UInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ - M(Bool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ + M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ \ M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ From adf5d24177b6d23d4788e531fa2267378c07aae6 Mon Sep 17 00:00:00 2001 From: M0r64n Date: Thu, 18 Feb 2021 11:36:17 +0400 Subject: [PATCH 03/48] Correct file engine settings tests --- .../01720_engine_file_empty_if_not_exists.sql | 1 + .../01721_engine_file_truncate_on_insert.sql | 21 ++++++++++--------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql b/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql index c04e01ccc88..d665dbc722f 100644 --- a/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql +++ b/tests/queries/0_stateless/01720_engine_file_empty_if_not_exists.sql @@ -13,3 +13,4 @@ SET engine_file_empty_if_not_exists=1; SELECT * FROM file_engine_table; SET engine_file_empty_if_not_exists=0; +DROP TABLE file_engine_table; diff --git a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql index 65246db7963..42d935cc0dd 100644 --- a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql +++ b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql @@ -1,20 +1,21 @@ -INSERT INTO TABLE FUNCTION file('01718_file/test/data.TSV', 'TSV', 'id UInt32') VALUES ('file', 42); +DROP TABLE IF EXISTS test; + +INSERT INTO TABLE FUNCTION file('01718_file/test/data.TSV', 'TSV', 'id UInt32') VALUES (1); ATTACH TABLE test FROM '01718_file/test' (id UInt8) ENGINE=File(TSV); -CREATE TABLE file_engine_table (id UInt32) ENGINE=File(TabSeparated); - -INSERT INTO file_engine_table VALUES (1), (2), (3); -INSERT INTO file_engine_table VALUES (4); -SELECT * FROM file_engine_table; +INSERT INTO test VALUES (2), (3); +INSERT INTO test VALUES (4); +SELECT * FROM test; SET engine_file_truncate_on_insert=0; -INSERT INTO file_engine_table VALUES (5), (6); -SELECT * FROM file_engine_table; +INSERT INTO test VALUES (5), (6); +SELECT * FROM test; SET engine_file_truncate_on_insert=1; -INSERT INTO file_engine_table VALUES (0), (1), (2); -SELECT * FROM file_engine_table; +INSERT INTO test VALUES (0), (1), (2); +SELECT * FROM test; SET engine_file_truncate_on_insert=0; +DROP TABLE test; From 1ce9570fcb4919880c19b05986dd9f7691fefb6f Mon Sep 17 00:00:00 2001 From: M0r64n Date: Thu, 18 Feb 2021 07:50:15 +0000 Subject: [PATCH 04/48] Fix 01721_engine_file_truncate_on_insert.reference --- .../0_stateless/01721_engine_file_truncate_on_insert.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference index a25fb4f0e7e..578661c9194 100644 --- a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference +++ b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.reference @@ -10,4 +10,4 @@ 6 0 1 -2 \ No newline at end of file +2 From 77fd060665751fc6528dd9f77e0fdea41cbc23bc Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 14 Feb 2021 19:09:36 +0800 Subject: [PATCH 05/48] 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 06/48] 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 07/48] 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 08/48] 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 09/48] 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 10/48] 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 11/48] 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 2f7d0ba92677f595b1d760af2a826cc6fa181802 Mon Sep 17 00:00:00 2001 From: M0r64n Date: Sat, 20 Feb 2021 03:27:23 +0400 Subject: [PATCH 12/48] Replace direct truncate with O_TRUNC flag --- src/Storages/StorageFile.cpp | 16 ++++++++++------ .../01721_engine_file_truncate_on_insert.sql | 4 ++-- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 856d03ea2ce..5524569e1f0 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -475,7 +475,8 @@ public: std::unique_lock && lock_, const CompressionMethod compression_method, const Context & context, - const std::optional & format_settings) + const std::optional & format_settings, + int & flags) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(std::move(lock_)) @@ -491,13 +492,14 @@ public: * INSERT data; SELECT *; last SELECT returns only insert_data */ storage.table_fd_was_used = true; - naked_buffer = std::make_unique(storage.table_fd); + naked_buffer = std::make_unique(storage.table_fd, DBMS_DEFAULT_BUFFER_SIZE); } else { if (storage.paths.size() != 1) throw Exception("Table '" + storage.getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); - naked_buffer = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + flags |= O_WRONLY | O_APPEND | O_CREAT; + naked_buffer = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags); } /// In case of CSVWithNames we have already written prefix. @@ -552,10 +554,11 @@ BlockOutputStreamPtr StorageFile::write( if (format_name == "Distributed") throw Exception("Method write is not implemented for Distributed format", ErrorCodes::NOT_IMPLEMENTED); + int flags = 0; + std::string path; if (context.getSettingsRef().engine_file_truncate_on_insert) - if (0 != ::truncate(paths[0].c_str(), 0)) - throwFromErrnoWithPath("Cannot truncate file " + paths[0], paths[0], ErrorCodes::CANNOT_TRUNCATE_FILE); + flags |= O_TRUNC; if (!paths.empty()) { @@ -569,7 +572,8 @@ BlockOutputStreamPtr StorageFile::write( std::unique_lock{rwlock, getLockTimeout(context)}, chooseCompressionMethod(path, compression_method), context, - format_settings); + format_settings, + flags); } bool StorageFile::storesDataOnDisk() const diff --git a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql index 42d935cc0dd..079b2546a20 100644 --- a/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql +++ b/tests/queries/0_stateless/01721_engine_file_truncate_on_insert.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS test; -INSERT INTO TABLE FUNCTION file('01718_file/test/data.TSV', 'TSV', 'id UInt32') VALUES (1); -ATTACH TABLE test FROM '01718_file/test' (id UInt8) ENGINE=File(TSV); +INSERT INTO TABLE FUNCTION file('01721_file/test/data.TSV', 'TSV', 'id UInt32') VALUES (1); +ATTACH TABLE test FROM '01721_file/test' (id UInt8) ENGINE=File(TSV); INSERT INTO test VALUES (2), (3); INSERT INTO test VALUES (4); From 7c04f15c8031a63f20573b9948dd18005f860f26 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 09:11:42 +0300 Subject: [PATCH 13/48] 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 14/48] 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 15/48] 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 64e76a4a8da87adb374ffeb571fe76eac4850ae8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Feb 2021 21:13:36 +0300 Subject: [PATCH 16/48] Minor changes in Decimal --- src/Core/DecimalComparison.h | 2 +- src/Core/DecimalFunctions.h | 24 +++++++++---------- src/Core/MySQL/MySQLReplication.cpp | 6 ++--- src/DataTypes/DataTypeDateTime64.cpp | 4 ++-- src/DataTypes/DataTypeDecimalBase.h | 10 ++++---- src/DataTypes/DataTypesDecimal.cpp | 2 +- src/DataTypes/DataTypesDecimal.h | 2 +- src/DataTypes/convertMySQLDataType.cpp | 6 ++--- .../fetchPostgreSQLTableStructure.cpp | 8 +++---- src/Functions/array/arrayAggregation.cpp | 2 +- src/Functions/array/arrayCumSum.cpp | 2 +- .../array/arrayCumSumNonNegative.cpp | 2 +- src/Functions/isDecimalOverflow.cpp | 2 +- src/IO/WriteHelpers.h | 20 ++++++++-------- 14 files changed, 46 insertions(+), 46 deletions(-) diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 8279d01d35a..486c2c1f8f4 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -78,7 +78,7 @@ public: static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b) { - static const UInt32 max_scale = DecimalUtils::maxPrecision(); + static const UInt32 max_scale = DecimalUtils::max_precision; if (scale_a > max_scale || scale_b > max_scale) throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW); diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 2b916cbf538..355cf1d378a 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -24,13 +24,13 @@ namespace ErrorCodes namespace DecimalUtils { -static constexpr size_t minPrecision() { return 1; } -template static constexpr size_t maxPrecision() { return 0; } -template <> constexpr size_t maxPrecision() { return 9; } -template <> constexpr size_t maxPrecision() { return 18; } -template <> constexpr size_t maxPrecision() { return 18; } -template <> constexpr size_t maxPrecision() { return 38; } -template <> constexpr size_t maxPrecision() { return 76; } +inline constexpr size_t min_precision = 1; +template inline constexpr size_t max_precision = 0; +template <> inline constexpr size_t max_precision = 9; +template <> inline constexpr size_t max_precision = 18; +template <> inline constexpr size_t max_precision = 18; +template <> inline constexpr size_t max_precision = 38; +template <> inline constexpr size_t max_precision = 76; template inline auto scaleMultiplier(UInt32 scale) @@ -87,7 +87,7 @@ struct DataTypeDecimalTrait * * Sign of `whole` controls sign of result: negative whole => negative result, positive whole => positive result. * Sign of `fractional` is expected to be positive, otherwise result is undefined. - * If `scale` is to big (scale > maxPrecision), result is undefined. + * If `scale` is to big (scale > max_precision), result is undefined. */ template inline DecimalType decimalFromComponentsWithMultiplier( @@ -287,21 +287,21 @@ inline auto binaryOpResult(const DecimalType & tx, const DecimalType & ty) scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); if constexpr (sizeof(T) < sizeof(U)) - return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), scale); + return DataTypeDecimalTrait(DecimalUtils::max_precision, scale); else - return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), scale); + return DataTypeDecimalTrait(DecimalUtils::max_precision, scale); } template typename DecimalType> inline const DataTypeDecimalTrait binaryOpResult(const DecimalType & tx, const DataTypeNumber &) { - return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), tx.getScale()); + return DataTypeDecimalTrait(DecimalUtils::max_precision, tx.getScale()); } template typename DecimalType> inline const DataTypeDecimalTrait binaryOpResult(const DataTypeNumber &, const DecimalType & ty) { - return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), ty.getScale()); + return DataTypeDecimalTrait(DecimalUtils::max_precision, ty.getScale()); } } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 8fdf337c849..1b202c4edb4 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -475,11 +475,11 @@ namespace MySQLReplication { const auto & dispatch = [](const size_t & precision, const size_t & scale, const auto & function) -> Field { - if (precision <= DecimalUtils::maxPrecision()) + if (precision <= DecimalUtils::max_precision) return Field(function(precision, scale, Decimal32())); - else if (precision <= DecimalUtils::maxPrecision()) + else if (precision <= DecimalUtils::max_precision) return Field(function(precision, scale, Decimal64())); - else if (precision <= DecimalUtils::maxPrecision()) + else if (precision <= DecimalUtils::max_precision) return Field(function(precision, scale, Decimal128())); return Field(function(precision, scale, Decimal256())); diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index 09e39c2de1a..17b94e871bf 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes static constexpr UInt32 max_scale = 9; DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name) - : DataTypeDecimalBase(DecimalUtils::maxPrecision(), scale_), + : DataTypeDecimalBase(DecimalUtils::max_precision, scale_), TimezoneMixin(time_zone_name) { if (scale > max_scale) @@ -37,7 +37,7 @@ DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_z } DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info) - : DataTypeDecimalBase(DecimalUtils::maxPrecision(), scale_), + : DataTypeDecimalBase(DecimalUtils::max_precision, scale_), TimezoneMixin(time_zone_info) { if (scale > max_scale) diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index c861b3bcac0..d9079166fa7 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -65,7 +65,7 @@ public: static constexpr bool is_parametric = true; - static constexpr size_t maxPrecision() { return DecimalUtils::maxPrecision(); } + static constexpr size_t maxPrecision() { return DecimalUtils::max_precision; } DataTypeDecimalBase(UInt32 precision_, UInt32 scale_) : precision(precision_), @@ -197,17 +197,17 @@ inline const DecimalType decimalResultType(const DataTypeNumber & tx, cons template