From 7990c8cbad00f4287374573c6020208b21d3b556 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 Nov 2020 22:50:01 +0300 Subject: [PATCH 01/34] fix optimization with 'optimize_aggregators_of_group_by_keys' and joins --- .../AggregateFunctionOfGroupByKeysVisitor.h | 26 +++--- src/Interpreters/TreeOptimizer.cpp | 81 +++++-------------- ...egate_functions_of_key_with_join.reference | 1 + ...1_aggregate_functions_of_key_with_join.sql | 5 ++ 4 files changed, 36 insertions(+), 77 deletions(-) create mode 100644 tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference create mode 100644 tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index 6b903ec45cf..587baa660cb 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -20,8 +20,8 @@ struct KeepAggregateFunctionMatcher { struct Data { - std::unordered_set & group_by_keys; - bool & keep_aggregator; + const NameSet & group_by_keys; + bool keep_aggregator; }; using Visitor = InDepthNodeVisitor; @@ -33,7 +33,7 @@ struct KeepAggregateFunctionMatcher static void visit(ASTFunction & function_node, Data & data) { - if ((function_node.arguments->children).empty()) + if (function_node.arguments->children.empty()) { data.keep_aggregator = true; return; @@ -47,12 +47,9 @@ struct KeepAggregateFunctionMatcher static void visit(ASTIdentifier & ident, Data & data) { - if (!data.group_by_keys.count(ident.shortName())) - { - /// if variable of a function is not in GROUP BY keys, this function should not be deleted + /// if variable of a function is not in GROUP BY keys, this function should not be deleted + if (!data.group_by_keys.count(ident.getColumnName())) data.keep_aggregator = true; - return; - } } static void visit(const ASTPtr & ast, Data & data) @@ -75,21 +72,21 @@ struct KeepAggregateFunctionMatcher } }; -using KeepAggregateFunctionVisitor = InDepthNodeVisitor; +using KeepAggregateFunctionVisitor = KeepAggregateFunctionMatcher::Visitor; class SelectAggregateFunctionOfGroupByKeysMatcher { public: struct Data { - std::unordered_set & group_by_keys; + const NameSet & group_by_keys; }; static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { /// Don't descent into table functions and subqueries and special case for ArrayJoin. - return !node->as() && - !(node->as() || node->as() || node->as()); + return !node->as() && !node->as() + && !node->as() && !node->as(); } static void visit(ASTPtr & ast, Data & data) @@ -99,12 +96,11 @@ public: if (function_node && (function_node->name == "min" || function_node->name == "max" || function_node->name == "any" || function_node->name == "anyLast")) { - bool keep_aggregator = false; - KeepAggregateFunctionVisitor::Data keep_data{data.group_by_keys, keep_aggregator}; + KeepAggregateFunctionVisitor::Data keep_data{data.group_by_keys, false}; KeepAggregateFunctionVisitor(keep_data).visit(function_node->arguments); /// Place argument of an aggregate function instead of function - if (!keep_aggregator) + if (!keep_data.keep_aggregator) { String alias = function_node->alias; ast = (function_node->arguments->children[0])->clone(); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 61ca933dd53..b0f9ef187f1 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -177,43 +177,21 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum struct GroupByKeysInfo { - std::unordered_set key_names; ///set of keys' short names - bool has_identifier = false; + NameSet key_names; ///set of keys' short names bool has_function = false; - bool has_possible_collision = false; }; -GroupByKeysInfo getGroupByKeysInfo(ASTs & group_keys) +GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) { GroupByKeysInfo data; - ///filling set with short names of keys - for (auto & group_key : group_keys) + /// filling set with short names of keys + for (auto & group_key : group_by_keys) { if (group_key->as()) data.has_function = true; - if (auto * group_key_ident = group_key->as()) - { - data.has_identifier = true; - if (data.key_names.count(group_key_ident->shortName())) - { - ///There may be a collision between different tables having similar variables. - ///Due to the fact that we can't track these conflicts yet, - ///it's better to disable some optimizations to avoid elimination necessary keys. - data.has_possible_collision = true; - } - - data.key_names.insert(group_key_ident->shortName()); - } - else if (auto * group_key_func = group_key->as()) - { - data.key_names.insert(group_key_func->getColumnName()); - } - else - { - data.key_names.insert(group_key->getColumnName()); - } + data.key_names.insert(group_key->getColumnName()); } return data; @@ -225,47 +203,28 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) if (!select_query->groupBy()) return; - auto grp_by = select_query->groupBy(); - auto & group_keys = grp_by->children; + auto group_by = select_query->groupBy(); + const auto & group_by_keys = group_by->children; ASTs modified; ///result - GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_keys); + GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_by_keys); - if (!group_by_keys_data.has_function || group_by_keys_data.has_possible_collision) + if (!group_by_keys_data.has_function) return; GroupByFunctionKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; - GroupByFunctionKeysVisitor(visitor_data).visit(grp_by); + GroupByFunctionKeysVisitor(visitor_data).visit(group_by); - modified.reserve(group_keys.size()); + modified.reserve(group_by_keys.size()); - ///filling the result - for (auto & group_key : group_keys) - { - if (auto * group_key_func = group_key->as()) - { - if (group_by_keys_data.key_names.count(group_key_func->getColumnName())) - modified.push_back(group_key); + /// filling the result + for (auto & group_key : group_by_keys) + if (group_by_keys_data.key_names.count(group_key->getColumnName())) + modified.push_back(group_key); - continue; - } - if (auto * group_key_ident = group_key->as()) - { - if (group_by_keys_data.key_names.count(group_key_ident->shortName())) - modified.push_back(group_key); - - continue; - } - else - { - if (group_by_keys_data.key_names.count(group_key->getColumnName())) - modified.push_back(group_key); - } - } - - ///modifying the input - grp_by->children = modified; + /// modifying the input + group_by->children = modified; } /// Eliminates min/max/any-aggregators of functions of GROUP BY keys @@ -274,10 +233,8 @@ void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTP if (!select_query->groupBy()) return; - auto grp_by = select_query->groupBy(); - auto & group_keys = grp_by->children; - - GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_keys); + auto & group_by_keys = select_query->groupBy()->children; + GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_by_keys); SelectAggregateFunctionOfGroupByKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; SelectAggregateFunctionOfGroupByKeysVisitor(visitor_data).visit(node); diff --git a/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference new file mode 100644 index 00000000000..9874d6464ab --- /dev/null +++ b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference @@ -0,0 +1 @@ +1 2 diff --git a/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql new file mode 100644 index 00000000000..66047fcc1a6 --- /dev/null +++ b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql @@ -0,0 +1,5 @@ +SET optimize_aggregators_of_group_by_keys = 1; +SELECT source.key, max(target.key) FROM (SELECT 1 key, 'x' name) source +INNER JOIN (SELECT 2 key, 'x' name) target +ON source.name = target.name +GROUP BY source.key; From f12923313d2fab98374f989d8484b8e507f4e172 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 13 Nov 2020 00:01:53 +0300 Subject: [PATCH 02/34] fix clang-tidy --- src/Interpreters/TreeOptimizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index b0f9ef187f1..0307095767e 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -219,7 +219,7 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) modified.reserve(group_by_keys.size()); /// filling the result - for (auto & group_key : group_by_keys) + for (const auto & group_key : group_by_keys) if (group_by_keys_data.key_names.count(group_key->getColumnName())) modified.push_back(group_key); From a03b6df897a8506025675e585be3f7e136b0e764 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 13 Nov 2020 02:33:26 +0300 Subject: [PATCH 03/34] fix order by optimization with monotonous functions --- src/Interpreters/TreeOptimizer.cpp | 31 +++++++++---- src/Interpreters/TreeOptimizer.h | 5 +- src/Interpreters/TreeRewriter.cpp | 2 +- ...monotonous_functions_in_order_by.reference | 12 ++--- ...monotonous_functions_in_order_by.reference | 46 +++++++++++++++++++ ...imize_monotonous_functions_in_order_by.sql | 26 +++++++++++ 6 files changed, 103 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference create mode 100644 tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 61ca933dd53..603733129f4 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -26,6 +26,7 @@ #include #include +#include namespace DB @@ -438,7 +439,8 @@ void optimizeDuplicateDistinct(ASTSelectQuery & select) /// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression, /// has a single argument and not an aggregate functions. void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const Context & context, - const TablesWithColumns & tables_with_columns) + const TablesWithColumns & tables_with_columns, + const Names & sorting_key_columns) { auto order_by = select_query->orderBy(); if (!order_by) @@ -455,13 +457,22 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const C } } - for (auto & child : order_by->children) + bool is_sorting_key_prefix = true; + for (size_t i = 0; i < order_by->children.size(); ++i) { - auto * order_by_element = child->as(); + auto * order_by_element = order_by->children[i]->as(); auto & ast_func = order_by_element->children[0]; if (!ast_func->as()) continue; + if (i >= sorting_key_columns.size() || ast_func->getColumnName() != sorting_key_columns[i]) + is_sorting_key_prefix = false; + + /// If order by expression matches the sorting key, do not remove + /// functions to allow execute reading in order of key. + if (is_sorting_key_prefix) + continue; + MonotonicityCheckVisitor::Data data{tables_with_columns, context, group_by_hashes}; MonotonicityCheckVisitor(data).visit(ast_func); @@ -611,7 +622,8 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - const Context & context, bool & rewrite_subqueries) + const Context & context, const StorageMetadataPtr & metadata_snapshot, + bool & rewrite_subqueries) { const auto & settings = context.getSettingsRef(); @@ -652,9 +664,6 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou optimizeAggregateFunctionsOfGroupByKeys(select_query, query); } - /// Remove duplicate items from ORDER BY. - optimizeDuplicatesInOrderBy(select_query); - /// Remove duplicate ORDER BY and DISTINCT from subqueries. if (settings.optimize_duplicate_order_by_and_distinct) { @@ -672,7 +681,13 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou /// Replace monotonous functions with its argument if (settings.optimize_monotonous_functions_in_order_by) - optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns); + optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns, + metadata_snapshot ? metadata_snapshot->getSortingKeyColumns() : Names{}); + + /// Remove duplicate items from ORDER BY. + /// Execute it after all order by optimizations, + /// because they can produce duplicated columns. + optimizeDuplicatesInOrderBy(select_query); /// If function "if" has String-type arguments, transform them into enum if (settings.optimize_if_transform_strings_to_enum) diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index a81264184c1..a10dfc57451 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -8,6 +8,8 @@ namespace DB { class Context; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; /// Part of of Tree Rewriter (SyntaxAnalyzer) that optimizes AST. /// Query should be ready to execute either before either after it. But resulting query could be faster. @@ -16,7 +18,8 @@ class TreeOptimizer public: static void apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - const Context & context, bool & rewrite_subqueries); + const Context & context, const StorageMetadataPtr & metadata_snapshot, + bool & rewrite_subqueries); static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b2dbd027191..c02d54435eb 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -645,7 +645,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, context, subquery_depth, result.scalars, select_options.only_analyze); - TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.rewrite_subqueries); + TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.metadata_snapshot, result.rewrite_subqueries); /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); diff --git a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference index ffa91586f35..0a7746e1832 100644 --- a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference @@ -54,14 +54,10 @@ FROM numbers(3) ORDER BY exp(number) ASC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY - number ASC, - number ASC +ORDER BY number ASC SELECT number AS x FROM numbers(3) -ORDER BY - number ASC, - number ASC +ORDER BY number ASC SELECT number FROM numbers(3) ORDER BY number DESC @@ -79,9 +75,7 @@ FROM numbers(3) ORDER BY exp(number) DESC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY - number DESC, - number DESC +ORDER BY number DESC 0 1 2 diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference new file mode 100644 index 00000000000..203efdab299 --- /dev/null +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -0,0 +1,46 @@ +SELECT + timestamp, + key +FROM test_order_by +ORDER BY timestamp ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY toDate(timestamp) ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + FinishSorting + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree with order) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY + toDate(timestamp) ASC, + timestamp ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + FinishSorting + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree with order) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY timestamp ASC +LIMIT 10 diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql new file mode 100644 index 00000000000..b31457d8f68 --- /dev/null +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql @@ -0,0 +1,26 @@ +SET optimize_monotonous_functions_in_order_by = 1; + +DROP TABLE IF EXISTS test_order_by; + +CREATE TABLE test_order_by (timestamp DateTime, key UInt32) ENGINE=MergeTree() ORDER BY (toDate(timestamp), key); +INSERT INTO test_order_by SELECT now() + toIntervalSecond(number), number % 4 FROM numbers(10000); +OPTIMIZE TABLE test_order_by FINAL; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY timestamp LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY timestamp LIMIT 10; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp) LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY toDate(timestamp) LIMIT 10; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; + +DROP TABLE IF EXISTS test_order_by; + +CREATE TABLE test_order_by (timestamp DateTime, key UInt32) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_order_by SELECT now() + toIntervalSecond(number), number % 4 FROM numbers(10000); +OPTIMIZE TABLE test_order_by FINAL; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; + +DROP TABLE IF EXISTS test_order_by; From 1e24d6085a596abb8c56213b7699e67547860dfc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 13 Nov 2020 18:56:25 +0300 Subject: [PATCH 04/34] fix performance with Merge table over huge number of MergeTree tables --- src/Interpreters/ExpressionAnalyzer.cpp | 5 ----- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/TreeRewriter.cpp | 13 +++++++++++++ src/Interpreters/TreeRewriter.h | 11 ++++------- 4 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 45230c53e81..fe8866f6c70 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -139,11 +139,6 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(); } -bool ExpressionAnalyzer::isRemoteStorage() const -{ - return storage() && storage()->isRemote(); -} - void ExpressionAnalyzer::analyzeAggregation() { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index bd027e5a613..1b94e5c91f9 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -163,7 +163,7 @@ protected: const ASTSelectQuery * getSelectQuery() const; - bool isRemoteStorage() const; + bool isRemoteStorage() const { return syntax->is_remote_storage; } }; class SelectQueryExpressionAnalyzer; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b2dbd027191..fb3495ca873 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -387,6 +387,19 @@ std::vector getAggregates(ASTPtr & query, const ASTSelectQu } +TreeRewriterResult::TreeRewriterResult( + const NamesAndTypesList & source_columns_, + ConstStoragePtr storage_, + const StorageMetadataPtr & metadata_snapshot_, + bool add_special) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , source_columns(source_columns_) +{ + collectSourceColumns(add_special); + is_remote_storage = storage && storage->isRemote(); +} + /// Add columns from storage to source_columns list. Deduplicate resulted list. /// Special columns are non physical columns, for example ALIAS void TreeRewriterResult::collectSourceColumns(bool add_special) diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 83cfabe2ec4..4075be4326d 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -53,6 +53,9 @@ struct TreeRewriterResult bool optimize_trivial_count = false; + /// Cache isRemote() call for storage, because it may be too heavy. + bool is_remote_storage = false; + /// Results of scalar sub queries Scalars scalars; @@ -60,13 +63,7 @@ struct TreeRewriterResult const NamesAndTypesList & source_columns_, ConstStoragePtr storage_ = {}, const StorageMetadataPtr & metadata_snapshot_ = {}, - bool add_special = true) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , source_columns(source_columns_) - { - collectSourceColumns(add_special); - } + bool add_special = true); void collectSourceColumns(bool add_special); void collectUsedColumns(const ASTPtr & query, bool is_select); From 6f54cc99d6d36d39568d90ff8c129bc00faa95b0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 13 Nov 2020 19:30:10 +0300 Subject: [PATCH 05/34] fix clang-tidy --- src/Interpreters/TreeOptimizer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 0307095767e..4350ebdbe34 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -186,7 +186,7 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) GroupByKeysInfo data; /// filling set with short names of keys - for (auto & group_key : group_by_keys) + for (const auto & group_key : group_by_keys) { if (group_key->as()) data.has_function = true; @@ -233,7 +233,7 @@ void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTP if (!select_query->groupBy()) return; - auto & group_by_keys = select_query->groupBy()->children; + const auto & group_by_keys = select_query->groupBy()->children; GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_by_keys); SelectAggregateFunctionOfGroupByKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; From b10283f8d5d487ff21eb8daa9a4eb2e11e3cd08c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Nov 2020 12:32:57 +0300 Subject: [PATCH 06/34] Add a test for #13990 --- ...ibuted_buffer_cannot_find_column.reference | 3 +++ ..._distributed_buffer_cannot_find_column.sql | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.reference create mode 100644 tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql diff --git a/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.reference b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql new file mode 100644 index 00000000000..626f55d849a --- /dev/null +++ b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS realtimedrep; +DROP TABLE IF EXISTS realtimedistributed; +DROP TABLE IF EXISTS realtimebuff; + +CREATE TABLE realtimedrep(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = ReplicatedMergeTree('/data/clickhouse/3/realtimedrep','2') PARTITION BY toDate(reqDate) ORDER BY transID SETTINGS index_granularity = 8192; +CREATE TABLE realtimedistributed(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), realtimedrep, rand()); +CREATE TABLE realtimebuff(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = Buffer(currentDatabase(), 'realtimedistributed', 16, 3600, 36000, 10000, 1000000, 10000000, 100000000); + +insert into realtimebuff (amount,transID,userID,appID,appName,transType,orderSource,nau,fau,transactionType,supplier,fMerchant,bankConnCode,reqDate) values (100, '200312000295032','200223000028708','14', 'Data','1', '20','1', '0','123','abc', '1234a','ZPVBIDV', 1598256583); + +select sum(amount) = 100 from realtimebuff; + +OPTIMIZE TABLE realtimebuff; +select sum(amount) IN (100, 200) from realtimebuff; + +SYSTEM FLUSH DISTRIBUTED realtimedistributed; +select sum(amount) = 200 from realtimebuff; + +DROP TABLE realtimedrep; +DROP TABLE realtimedistributed; +DROP TABLE realtimebuff; From 24710046c32bf8b1e20a9ccc1f955bdddbbddd28 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Nov 2020 12:34:23 +0300 Subject: [PATCH 07/34] Arcadia does not support distributed tests --- tests/queries/0_stateless/arcadia_skip_list.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6875151520a..21ce7b5b5de 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -166,7 +166,8 @@ 01548_query_log_query_execution_ms 01552_dict_fixedstring 01555_system_distribution_queue_mask -01557_max_parallel_replicas_no_sample.sql +01557_max_parallel_replicas_no_sample 01525_select_with_offset_fetch_clause 01560_timeseriesgroupsum_segfault 00976_ttl_with_old_parts +01584_distributed_buffer_cannot_find_column From ed679d8edbefde595312a18b1686c6ea6fee9177 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Nov 2020 12:38:03 +0300 Subject: [PATCH 08/34] Better test --- .../0_stateless/01584_distributed_buffer_cannot_find_column.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql index 626f55d849a..3ba6a1c9528 100644 --- a/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql +++ b/tests/queries/0_stateless/01584_distributed_buffer_cannot_find_column.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS realtimedrep; DROP TABLE IF EXISTS realtimedistributed; DROP TABLE IF EXISTS realtimebuff; -CREATE TABLE realtimedrep(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = ReplicatedMergeTree('/data/clickhouse/3/realtimedrep','2') PARTITION BY toDate(reqDate) ORDER BY transID SETTINGS index_granularity = 8192; +CREATE TABLE realtimedrep(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = MergeTree PARTITION BY toDate(reqDate) ORDER BY transID SETTINGS index_granularity = 8192; CREATE TABLE realtimedistributed(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), realtimedrep, rand()); CREATE TABLE realtimebuff(amount Int64,transID String,userID String,appID String,appName String,transType String,orderSource String,nau String,fau String,transactionType String,supplier String,fMerchant String,bankConnCode String,reqDate DateTime) ENGINE = Buffer(currentDatabase(), 'realtimedistributed', 16, 3600, 36000, 10000, 1000000, 10000000, 100000000); From 77c963dec5104e2e70ac9a6d955b62067e5ad050 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Nov 2020 13:32:56 +0300 Subject: [PATCH 09/34] Minor changed for ODBC storage --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 00ca89bd887..4cbd60474c9 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -79,10 +79,13 @@ namespace assert_cast(column).insert(value.convert()); break; case ValueType::vtDate: - assert_cast(column).insertValue(UInt16{LocalDate{value.convert()}.getDayNum()}); + Poco::DateTime date = value.convert(); + assert_cast(column).insertValue(UInt16{LocalDate(date.year(), date.month(), date.day()).getDayNum()}); break; case ValueType::vtDateTime: - assert_cast(column).insertValue(time_t{LocalDateTime{value.convert()}}); + Poco::DateTime datetime = value.convert(); + assert_cast(column).insertValue(time_t{LocalDateTime( + datetime.year(), datetime.month(), datetime.day(), datetime.hour(), datetime.minute(), datetime.second())}); break; case ValueType::vtUUID: assert_cast(column).insert(parse(value.convert())); @@ -112,6 +115,7 @@ Block ODBCBlockInputStream::readImpl() for (const auto idx : ext::range(0, row.fieldCount())) { + /// TODO This is extremely slow. const Poco::Dynamic::Var & value = row[idx]; if (!value.isEmpty()) From 8eccf680d8d6224134a6a9217255046cf4b8fb84 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 26 Nov 2020 07:11:34 +0300 Subject: [PATCH 10/34] Update ODBCBlockInputStream.cpp --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 4cbd60474c9..3e2a2d0c7d4 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -79,14 +79,18 @@ namespace assert_cast(column).insert(value.convert()); break; case ValueType::vtDate: + { Poco::DateTime date = value.convert(); assert_cast(column).insertValue(UInt16{LocalDate(date.year(), date.month(), date.day()).getDayNum()}); break; + } case ValueType::vtDateTime: + { Poco::DateTime datetime = value.convert(); assert_cast(column).insertValue(time_t{LocalDateTime( datetime.year(), datetime.month(), datetime.day(), datetime.hour(), datetime.minute(), datetime.second())}); break; + } case ValueType::vtUUID: assert_cast(column).insert(parse(value.convert())); break; From 022ba2b0a9b97bf1c4889ff47104c338afc4b502 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 18 Nov 2020 00:49:29 +0800 Subject: [PATCH 11/34] Fix unmatched type comparison in KeyCondition --- src/DataTypes/getLeastSupertype.cpp | 25 ++++++- src/Storages/MergeTree/KeyCondition.cpp | 68 ++++++++++++++----- ...9_date_datetime_index_comparison.reference | 5 ++ .../01579_date_datetime_index_comparison.sql | 16 +++++ 4 files changed, 95 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/01579_date_datetime_index_comparison.reference create mode 100644 tests/queries/0_stateless/01579_date_datetime_index_comparison.sql diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 43434081806..0d2d3e66200 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -358,6 +358,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) maximize(max_bits_of_unsigned_integer, 32); else if (typeid_cast(type.get())) maximize(max_bits_of_unsigned_integer, 64); + else if (typeid_cast(type.get())) + maximize(max_bits_of_unsigned_integer, 256); else if (typeid_cast(type.get())) maximize(max_bits_of_signed_integer, 8); else if (typeid_cast(type.get())) @@ -366,6 +368,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types) maximize(max_bits_of_signed_integer, 32); else if (typeid_cast(type.get())) maximize(max_bits_of_signed_integer, 64); + else if (typeid_cast(type.get())) + maximize(max_bits_of_signed_integer, 128); + else if (typeid_cast(type.get())) + maximize(max_bits_of_signed_integer, 256); else if (typeid_cast(type.get())) maximize(max_mantissa_bits_of_floating, 24); else if (typeid_cast(type.get())) @@ -386,7 +392,18 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// If unsigned is not covered by signed. if (max_bits_of_signed_integer && max_bits_of_unsigned_integer >= max_bits_of_signed_integer) - ++min_bit_width_of_integer; + { + // Because 128 and 256 bit integers are significantly slower, we should not promote to them. + // But if we already have wide numbers, promotion is necessary. + if (min_bit_width_of_integer != 64) + ++min_bit_width_of_integer; + else + throw Exception( + getExceptionMessagePrefix(types) + + " because some of them are signed integers and some are unsigned integers," + " but there is no signed integer type, that can exactly represent all required unsigned integer values", + ErrorCodes::NO_COMMON_TYPE); + } /// If the result must be floating. if (max_mantissa_bits_of_floating) @@ -413,6 +430,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return std::make_shared(); else if (min_bit_width_of_integer <= 64) return std::make_shared(); + else if (min_bit_width_of_integer <= 128) + return std::make_shared(); + else if (min_bit_width_of_integer <= 256) + return std::make_shared(); else throw Exception(getExceptionMessagePrefix(types) + " because some of them are signed integers and some are unsigned integers," @@ -429,6 +450,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return std::make_shared(); else if (min_bit_width_of_integer <= 64) return std::make_shared(); + else if (min_bit_width_of_integer <= 256) + return std::make_shared(); else throw Exception("Logical error: " + getExceptionMessagePrefix(types) + " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f6ff13dc9c6..ca5989fe32d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2,12 +2,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -23,7 +25,6 @@ #include #include - namespace DB { @@ -975,9 +976,6 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const ASTPtr & node) { - if (desired_type->equals(*src_type)) - return; - try { src_value = convertFieldToType(src_value, *desired_type, src_type.get()); @@ -1084,15 +1082,6 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont if (key_column_num == static_cast(-1)) throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); - /// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5" - if (is_constant_transformed) - { - if (func_name == "less") - func_name = "lessOrEquals"; - else if (func_name == "greater") - func_name = "greaterOrEquals"; - } - /// Replace on to <-sign> if (key_arg_pos == 1) { @@ -1114,12 +1103,55 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont } } - bool cast_not_needed = - is_set_const /// Set args are already casted inside Set::createFromAST - || (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast. + bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST + || ((isNativeNumber(key_expr_type) || isDateTime(key_expr_type)) + && (isNativeNumber(const_type) || isDateTime(const_type))); /// Numbers and DateTime are accurately compared without cast. + + if (!cast_not_needed && !key_expr_type->equals(*const_type)) + { + if (const_value.getType() == Field::Types::String) + { + const_value = convertFieldToType(const_value, *key_expr_type); + if (const_value.isNull()) + return false; + // No need to set is_constant_transformed because we're doing exact conversion + } + else + { + DataTypePtr common_type = getLeastSupertype({key_expr_type, const_type}); + if (!const_type->equals(*common_type)) + { + castValueToType(common_type, const_value, const_type, node); + + // Need to set is_constant_transformed unless we're doing exact conversion + if (!key_expr_type->equals(*common_type)) + is_constant_transformed = true; + } + if (!key_expr_type->equals(*common_type)) + { + ColumnsWithTypeAndName arguments{ + {nullptr, key_expr_type, ""}, {DataTypeString().createColumnConst(1, common_type->getName()), common_type, ""}}; + FunctionOverloadResolverPtr func_builder_cast + = std::make_shared(CastOverloadResolver::createImpl(false)); + auto func_cast = func_builder_cast->build(arguments); + + /// If we know the given range only contains one value, then we treat all functions as positive monotonic. + if (!func_cast || (!single_point && !func_cast->hasInformationAboutMonotonicity())) + return false; + chain.push_back(func_cast); + } + } + } + + /// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5" + if (is_constant_transformed) + { + if (func_name == "less") + func_name = "lessOrEquals"; + else if (func_name == "greater") + func_name = "greaterOrEquals"; + } - if (!cast_not_needed) - castValueToType(key_expr_type, const_value, const_type, node); } else return false; diff --git a/tests/queries/0_stateless/01579_date_datetime_index_comparison.reference b/tests/queries/0_stateless/01579_date_datetime_index_comparison.reference new file mode 100644 index 00000000000..e31666d96b0 --- /dev/null +++ b/tests/queries/0_stateless/01579_date_datetime_index_comparison.reference @@ -0,0 +1,5 @@ +1 +Array(Int64) +Array(Int128) +Array(Int128) +Array(UInt256) diff --git a/tests/queries/0_stateless/01579_date_datetime_index_comparison.sql b/tests/queries/0_stateless/01579_date_datetime_index_comparison.sql new file mode 100644 index 00000000000..60de837f8fc --- /dev/null +++ b/tests/queries/0_stateless/01579_date_datetime_index_comparison.sql @@ -0,0 +1,16 @@ +drop table if exists test_index; + +create table test_index(date Date) engine MergeTree partition by toYYYYMM(date) order by date; + +insert into test_index values('2020-10-30'); + +select 1 from test_index where date < toDateTime('2020-10-30 06:00:00'); + +drop table if exists test_index; + +select toTypeName([-1, toUInt32(1)]); +-- We don't promote to wide integers +select toTypeName([-1, toUInt64(1)]); -- { serverError 386 } +select toTypeName([-1, toInt128(1)]); +select toTypeName([toInt64(-1), toInt128(1)]); +select toTypeName([toUInt64(1), toUInt256(1)]); From 838596c7a4aa394c11edb9853957987d95a8fc77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Nov 2020 21:16:07 +0300 Subject: [PATCH 12/34] Implement countSubstrings() Function to count number of substring occurrences in the string: - in case of needle is multi char - counts non-intersecting substrings - the code is based on position helpers. The following new functions is available: - countSubstrings() - countSubstringsCaseInsensitive() - countSubstringsCaseInsensitiveUTF8() v0: substringCount() v2: - add substringCountCaseInsensitiveUTF8 - improve tests - fix coding style issues - fix multichar needle v3: rename to countSubstrings (by analogy with countEqual()) --- .../functions/string-search-functions.md | 54 ++++ src/Functions/CountSubstringsImpl.h | 232 ++++++++++++++++++ src/Functions/FunctionsStringSearch.h | 3 + src/Functions/countSubstrings.cpp | 24 ++ .../countSubstringsCaseInsensitive.cpp | 24 ++ .../countSubstringsCaseInsensitiveUTF8.cpp | 24 ++ .../registerFunctionsStringSearch.cpp | 8 + src/Functions/ya.make | 3 + .../01590_countSubstrings.reference | 111 +++++++++ .../0_stateless/01590_countSubstrings.sql | 138 +++++++++++ 10 files changed, 621 insertions(+) create mode 100644 src/Functions/CountSubstringsImpl.h create mode 100644 src/Functions/countSubstrings.cpp create mode 100644 src/Functions/countSubstringsCaseInsensitive.cpp create mode 100644 src/Functions/countSubstringsCaseInsensitiveUTF8.cpp create mode 100644 tests/queries/0_stateless/01590_countSubstrings.reference create mode 100644 tests/queries/0_stateless/01590_countSubstrings.sql diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 881139f103c..dba8a6e275c 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -536,4 +536,58 @@ For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCas !!! note "Note" For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables – collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function – we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one – this works for Latin and mostly for all Cyrillic letters. +## countSubstrings(haystack, needle) {#countSubstrings} + +Count the number of substring occurrences + +For a case-insensitive search, use the function `countSubstringsCaseInsensitive` (or `countSubstringsCaseInsensitiveUTF8`). + +**Syntax** + +``` sql +countSubstrings(haystack, needle[, start_pos]) +``` + +**Parameters** + +- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) + +**Returned values** + +- Number of occurrences. + +Type: `Integer`. + +**Examples** + +Query: + +``` sql +SELECT countSubstrings('foobar.com', '.') +``` + +Result: + +``` text +┌─countSubstrings('foobar.com', '.')─┐ +│ 1 │ +└────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT countSubstrings('aaaa', 'aa') +``` + +Result: + +``` text +┌─countSubstrings('aaaa', 'aa')─┐ +│ 2 │ +└───────────────────────────────┘ +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h new file mode 100644 index 00000000000..2c2ae20488f --- /dev/null +++ b/src/Functions/CountSubstringsImpl.h @@ -0,0 +1,232 @@ +#pragma once + +#include "PositionImpl.h" + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +/// Implementation of the countSubstrings() using helpers for position() +/// +/// NOTE: Intersecting substrings in haystack accounted only once, i.e.: +/// +/// countSubstrings('aaaa', 'aa') == 2 +template +struct CountSubstringsImpl +{ + static constexpr bool use_default_implementation_for_constants = false; + static constexpr bool supports_start_pos = true; + + using ResultType = UInt64; + + /// Count occurrences of one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + const std::string & needle, + const ColumnPtr & start_pos, + PaddedPODArray & res) + { + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// FIXME: suboptimal + memset(&res[0], 0, res.size() * sizeof(res[0])); + + /// Current index in the array of strings. + size_t i = 0; + + typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos); + + /// We will search for the next occurrence in all strings at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Determine which index it refers to. + while (begin + offsets[i] <= pos) + ++i; + + auto start = start_pos != nullptr ? start_pos->getUInt(i) : 0; + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + needle.size() < begin + offsets[i]) + { + auto res_pos = needle.size() + Impl::countChars(reinterpret_cast(begin + offsets[i - 1]), reinterpret_cast(pos)); + if (res_pos >= start) + { + ++res[i]; + } + /// Intersecting substrings in haystack accounted only once + pos += needle.size(); + continue; + } + pos = begin + offsets[i]; + ++i; + } + } + + /// Count number of occurrences of substring in string. + static void constantConstantScalar( + std::string data, + std::string needle, + UInt64 start_pos, + UInt64 & res) + { + res = 0; + + if (needle.size() == 0) + return; + + auto start = std::max(start_pos, UInt64(1)); + size_t start_byte = Impl::advancePos(data.data(), data.data() + data.size(), start - 1) - data.data(); + size_t new_start_byte; + while ((new_start_byte = data.find(needle, start_byte)) != std::string::npos) + { + ++res; + /// Intersecting substrings in haystack accounted only once + start_byte = new_start_byte + needle.size(); + } + } + + /// Count number of occurrences of substring in string starting from different positions. + static void constantConstant( + std::string data, + std::string needle, + const ColumnPtr & start_pos, + PaddedPODArray & res) + { + Impl::toLowerIfNeed(data); + Impl::toLowerIfNeed(needle); + + if (start_pos == nullptr) + { + constantConstantScalar(data, needle, 0, res[0]); + return; + } + + size_t haystack_size = Impl::countChars(data.data(), data.data() + data.size()); + + size_t size = start_pos != nullptr ? start_pos->size() : 0; + for (size_t i = 0; i < size; ++i) + { + auto start = start_pos->getUInt(i); + + if (start > haystack_size + 1) + { + res[i] = 0; + continue; + } + constantConstantScalar(data, needle, start, res[i]); + } + } + + /// Count number of occurrences of substring each time for a different inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & start_pos, + PaddedPODArray & res) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1); + + res[i] = 0; + if (start > haystack_size + 1) + { + /// 0 already + } + else if (0 == needle_size) + { + /// 0 already + } + else + { + /// It is assumed that the StringSearcher is not very difficult to initialize. + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( + reinterpret_cast(&needle_data[prev_needle_offset]), + needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end + + const UInt8 * end = reinterpret_cast(&haystack_data[haystack_offsets[i] - 1]); + const UInt8 * beg = reinterpret_cast(Impl::advancePos(reinterpret_cast(&haystack_data[prev_haystack_offset]), reinterpret_cast(end), start - 1)); + + const UInt8 * pos; + /// searcher returns a pointer to the found substring or to the end of `haystack`. + while ((pos = searcher.search(beg, end)) < end) + { + ++res[i]; + beg = pos + needle_size; + } + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Count number of substrings occurrences in the single string. + static void constantVector( + const String & haystack, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & start_pos, + PaddedPODArray & res) + { + /// NOTE You could use haystack indexing. But this is a rare case. + + ColumnString::Offset prev_needle_offset = 0; + + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1); + if (start <= haystack.size() + 1) + { + const char * needle_beg = reinterpret_cast(&needle_data[prev_needle_offset]); + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); + + const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); + const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); + + const UInt8 * pos; + while ((pos = searcher.search(beg, end)) < end) + { + ++res[i]; + beg = pos + needle_size; + } + } + + prev_needle_offset = needle_offsets[i]; + } + } + + template + static void vectorFixedConstant(Args &&...) + { + throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index a649c43b5a4..9e7d608aec6 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -29,6 +29,9 @@ namespace DB * multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns index of any match or zero if none; * multiMatchAllIndices(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns an array of matched indices in any order; * + * countSubstrings(haystack, needle) -- count number of occurences of needle in haystack. + * countSubstringsCaseInsensitive(haystack, needle) + * * Applies regexp re2 and pulls: * - the first subpattern, if the regexp has a subpattern; * - the zero subpattern (the match part, otherwise); diff --git a/src/Functions/countSubstrings.cpp b/src/Functions/countSubstrings.cpp new file mode 100644 index 00000000000..d0dd469b962 --- /dev/null +++ b/src/Functions/countSubstrings.cpp @@ -0,0 +1,24 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "CountSubstringsImpl.h" + + +namespace DB +{ +namespace +{ + +struct NameCountSubstrings +{ + static constexpr auto name = "countSubstrings"; +}; + +using FunctionCountSubstrings = FunctionsStringSearch, NameCountSubstrings>; + +} + +void registerFunctionCountSubstrings(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} +} diff --git a/src/Functions/countSubstringsCaseInsensitive.cpp b/src/Functions/countSubstringsCaseInsensitive.cpp new file mode 100644 index 00000000000..9207f2d4f20 --- /dev/null +++ b/src/Functions/countSubstringsCaseInsensitive.cpp @@ -0,0 +1,24 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "CountSubstringsImpl.h" + + +namespace DB +{ +namespace +{ + +struct NameCountSubstringsCaseInsensitive +{ + static constexpr auto name = "countSubstringsCaseInsensitive"; +}; + +using FunctionCountSubstringsCaseInsensitive = FunctionsStringSearch, NameCountSubstringsCaseInsensitive>; + +} + +void registerFunctionCountSubstringsCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp b/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..b1747bbd7ff --- /dev/null +++ b/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp @@ -0,0 +1,24 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "CountSubstringsImpl.h" + + +namespace DB +{ +namespace +{ + +struct NameCountSubstringsCaseInsensitiveUTF8 +{ + static constexpr auto name = "countSubstringsCaseInsensitiveUTF8"; +}; + +using FunctionCountSubstringsCaseInsensitiveUTF8 = FunctionsStringSearch, NameCountSubstringsCaseInsensitiveUTF8>; + +} + +void registerFunctionCountSubstringsCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/src/Functions/registerFunctionsStringSearch.cpp b/src/Functions/registerFunctionsStringSearch.cpp index e4f89201491..bdf94cf1134 100644 --- a/src/Functions/registerFunctionsStringSearch.cpp +++ b/src/Functions/registerFunctionsStringSearch.cpp @@ -31,6 +31,10 @@ void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory void registerFunctionHasToken(FunctionFactory &); void registerFunctionHasTokenCaseInsensitive(FunctionFactory &); +void registerFunctionCountSubstrings(FunctionFactory &); +void registerFunctionCountSubstringsCaseInsensitive(FunctionFactory &); +void registerFunctionCountSubstringsCaseInsensitiveUTF8(FunctionFactory &); + void registerFunctionsStringSearch(FunctionFactory & factory) { @@ -61,6 +65,10 @@ void registerFunctionsStringSearch(FunctionFactory & factory) registerFunctionHasToken(factory); registerFunctionHasTokenCaseInsensitive(factory); + + registerFunctionCountSubstrings(factory); + registerFunctionCountSubstringsCaseInsensitive(factory); + registerFunctionCountSubstringsCaseInsensitiveUTF8(factory); } } diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 5519e6e07c3..181cee53570 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -208,6 +208,9 @@ SRCS( cos.cpp cosh.cpp countDigits.cpp + countSubstrings.cpp + countSubstringsCaseInsensitive.cpp + countSubstringsCaseInsensitiveUTF8.cpp currentDatabase.cpp currentUser.cpp dateDiff.cpp diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference new file mode 100644 index 00000000000..056cb4c53b6 --- /dev/null +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -0,0 +1,111 @@ + +# countSubstrings + +CountSubstringsImpl::constantConstant +CountSubstringsImpl::constantConstantScalar +empty +0 +0 +0 +char +1 +2 +3 +word +1 +1 +1 +2 +3 +intersect +2 + +CountSubstringsImpl::vectorVector +1 +4 +6 +"intersect",4 + +CountSubstringsImpl::constantVector +2 +1 +0 +3 +5 +"intersect",4 + +CountSubstringsImpl::vectorConstant +0 +1 +2 +3 +4 +"intersect",4 + +# countSubstringsCaseInsensitive + +CountSubstringsImpl::constantConstant +CountSubstringsImpl::constantConstantScalar +char +1 +2 +3 +word +1 +1 +1 +2 +3 +intersect +2 + +CountSubstringsImpl::vectorVector +1 +3 +5 + +CountSubstringsImpl::constantVector +2 +1 +0 +3 +5 + +CountSubstringsImpl::vectorConstant +1 +0 +0 + +# countSubstringsCaseInsensitiveUTF8 + +CountSubstringsImpl::constantConstant +CountSubstringsImpl::constantConstantScalar +char +1 +2 +3 +word +1 +1 +1 +2 +3 +intersect +2 + +CountSubstringsImpl::vectorVector +1 +3 +5 +"intersect",4 + +CountSubstringsImpl::constantVector +2 +3 +5 +"intersect",4 + +CountSubstringsImpl::vectorConstant +1 +0 +"intersect",4 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql new file mode 100644 index 00000000000..dd2122ed6ff --- /dev/null +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -0,0 +1,138 @@ +-- +-- countSubstrings +-- +select ''; +select '# countSubstrings'; + +select ''; +select 'CountSubstringsImpl::constantConstant'; +select 'CountSubstringsImpl::constantConstantScalar'; + +select 'empty'; +select countSubstrings('', '.'); +select countSubstrings('', ''); +select countSubstrings('.', ''); + +select 'char'; +select countSubstrings('foobar.com', '.'); +select countSubstrings('www.foobar.com', '.'); +select countSubstrings('.foobar.com.', '.'); + +select 'word'; +select countSubstrings('foobar.com', 'com'); +select countSubstrings('com.foobar', 'com'); +select countSubstrings('foo.com.bar', 'com'); +select countSubstrings('com.foobar.com', 'com'); +select countSubstrings('com.foo.com.bar.com', 'com'); + +select 'intersect'; +select countSubstrings('aaaa', 'aa'); + +select ''; +select 'CountSubstringsImpl::vectorVector'; +select countSubstrings(toString(number), toString(number)) from numbers(1); +select countSubstrings(concat(toString(number), '000111'), toString(number)) from numbers(1); +select countSubstrings(concat(toString(number), '000111001'), toString(number)) from numbers(1); +select 'intersect', countSubstrings(concat(toString(number), '0000000'), '00') from numbers(1) format CSV; + +select ''; +select 'CountSubstringsImpl::constantVector'; +select countSubstrings('100', toString(number)) from numbers(3); +select countSubstrings('0100', toString(number)) from numbers(1); +select countSubstrings('010000', toString(number)) from numbers(1); +select 'intersect', countSubstrings('00000000', repeat(toString(number), 2)) from numbers(1) format CSV; + +select ''; +select 'CountSubstringsImpl::vectorConstant'; +select countSubstrings(toString(number), '1') from system.numbers limit 3 offset 9; +select countSubstrings(concat(toString(number), '000111'), '1') from numbers(1); +select countSubstrings(concat(toString(number), '000111001'), '1') from numbers(1); +select 'intersect', countSubstrings(repeat(toString(number), 8), '00') from numbers(1) format CSV; + +-- +-- countSubstringsCaseInsensitive +-- +select ''; +select '# countSubstringsCaseInsensitive'; + +select ''; +select 'CountSubstringsImpl::constantConstant'; +select 'CountSubstringsImpl::constantConstantScalar'; + +select 'char'; +select countSubstringsCaseInsensitive('aba', 'B'); +select countSubstringsCaseInsensitive('bab', 'B'); +select countSubstringsCaseInsensitive('BaBaB', 'b'); + +select 'word'; +select countSubstringsCaseInsensitive('foobar.com', 'COM'); +select countSubstringsCaseInsensitive('com.foobar', 'COM'); +select countSubstringsCaseInsensitive('foo.com.bar', 'COM'); +select countSubstringsCaseInsensitive('com.foobar.com', 'COM'); +select countSubstringsCaseInsensitive('com.foo.com.bar.com', 'COM'); + +select 'intersect'; +select countSubstringsCaseInsensitive('aaaa', 'AA'); + +select ''; +select 'CountSubstringsImpl::vectorVector'; +select countSubstringsCaseInsensitive(upper(char(number)), lower(char(number))) from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitive(concat(toString(number), 'aaa111'), char(number)) from numbers(100) where number = 0x41; +select countSubstringsCaseInsensitive(concat(toString(number), 'aaa111aa1'), char(number)) from numbers(100) where number = 0x41; + +select ''; +select 'CountSubstringsImpl::constantVector'; +select countSubstringsCaseInsensitive('aab', char(number)) from numbers(100) where number >= 0x41 and number <= 0x43; -- A..C +select countSubstringsCaseInsensitive('abaa', char(number)) from numbers(100) where number = 0x41; +select countSubstringsCaseInsensitive('abaaaa', char(number)) from numbers(100) where number = 0x41; + +select ''; +select 'CountSubstringsImpl::vectorConstant'; +select countSubstringsCaseInsensitive(char(number), 'a') from numbers(100) where number >= 0x41 and number <= 0x43; + +-- +-- countSubstringsCaseInsensitiveUTF8 +-- +select ''; +select '# countSubstringsCaseInsensitiveUTF8'; + +select ''; +select 'CountSubstringsImpl::constantConstant'; +select 'CountSubstringsImpl::constantConstantScalar'; + +select 'char'; +select countSubstringsCaseInsensitiveUTF8('фуу', 'Ф'); +select countSubstringsCaseInsensitiveUTF8('ФуФ', 'ф'); +select countSubstringsCaseInsensitiveUTF8('ФуФуФ', 'ф'); + +select 'word'; +select countSubstringsCaseInsensitiveUTF8('подстрока.рф', 'РФ'); +select countSubstringsCaseInsensitiveUTF8('рф.подстрока', 'рф'); +select countSubstringsCaseInsensitiveUTF8('подстрока.рф.подстрока', 'РФ'); +select countSubstringsCaseInsensitiveUTF8('рф.подстрока.рф', 'рф'); +select countSubstringsCaseInsensitiveUTF8('рф.подстрока.рф.подстрока.рф', 'РФ'); + +select 'intersect'; +select countSubstringsCaseInsensitiveUTF8('яяяя', 'ЯЯ'); + +select ''; +select 'CountSubstringsImpl::vectorVector'; +-- can't use any char, since this will not make valid UTF8 +-- for the haystack we use number as-is, for needle we just add dependency from number to go to vectorVector code +select countSubstringsCaseInsensitiveUTF8(upperUTF8(concat(char(number), 'я')), lowerUTF8(concat(substringUTF8(char(number), 2), 'Я'))) from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitiveUTF8(concat(toString(number), 'ЯЯЯ111'), concat(substringUTF8(char(number), 2), 'я')) from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitiveUTF8(concat(toString(number), 'яяя111яя1'), concat(substringUTF8(char(number), 2), 'Я')) from numbers(100) where number = 0x41; -- A +select 'intersect', countSubstringsCaseInsensitiveUTF8(concat(toString(number), 'яяяяяяяя'), concat(substringUTF8(char(number), 2), 'Яя')) from numbers(100) where number = 0x41 format CSV; -- A + +select ''; +select 'CountSubstringsImpl::constantVector'; +select countSubstringsCaseInsensitiveUTF8('ЯЯb', concat(substringUTF8(char(number), 2), 'я')) from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitiveUTF8('ЯbЯЯ', concat(substringUTF8(char(number), 2), 'я')) from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitiveUTF8('ЯbЯЯЯЯ', concat(substringUTF8(char(number), 2), 'я')) from numbers(100) where number = 0x41; -- A +select 'intersect', countSubstringsCaseInsensitiveUTF8('ЯЯЯЯЯЯЯЯ', concat(substringUTF8(char(number), 2), 'Яя')) from numbers(100) where number = 0x41 format CSV; -- A + +select ''; +select 'CountSubstringsImpl::vectorConstant'; +select countSubstringsCaseInsensitiveUTF8(concat(char(number), 'я'), 'Я') from numbers(100) where number = 0x41; -- A +select countSubstringsCaseInsensitiveUTF8(concat(char(number), 'б'), 'Я') from numbers(100) where number = 0x41; -- A +select 'intersect', countSubstringsCaseInsensitiveUTF8(concat(char(number), repeat('я', 8)), 'яЯ') from numbers(100) where number = 0x41 format CSV; -- A From 7c0138542b68b776bd670bb7a2dc8e831749cecb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Nov 2020 10:55:15 +0300 Subject: [PATCH 13/34] Query obfuscator: remove some words --- src/Parsers/obfuscateQueries.cpp | 20 ++++++++-------- .../01508_query_obfuscator.reference | 24 +++++++++---------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 11f4a77ee0e..eb0bf5281c9 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -62,7 +62,7 @@ std::initializer_list nouns "affinity", "afoul", "afterlife", "aftermath", "afternoon", "aftershave", "aftershock", "afterthought", "age", "agency", "agenda", "agent", "aggradation", "aggression", "aglet", "agony", "agreement", "agriculture", "aid", "aide", "aim", "air", "airbag", "airbus", "aircraft", "airfare", "airfield", "airforce", "airline", "airmail", "airman", "airplane", "airport", "airship", "airspace", "alarm", "alb", "albatross", -"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alias", "alibi", "alien", "allegation", "allergist", +"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alibi", "alien", "allegation", "allergist", "alley", "alliance", "alligator", "allocation", "allowance", "alloy", "alluvium", "almanac", "almighty", "almond", "alpaca", "alpenglow", "alpenhorn", "alpha", "alphabet", "altar", "alteration", "alternative", "altitude", "alto", "aluminium", "aluminum", "amazement", "amazon", "ambassador", "amber", "ambience", "ambiguity", "ambition", "ambulance", "amendment", "amenity", "ammunition", "amnesty", "amount", "amusement", @@ -124,7 +124,7 @@ std::initializer_list nouns "captain", "caption", "captor", "car", "carabao", "caramel", "caravan", "carbohydrate", "carbon", "carboxyl", "card", "cardboard", "cardigan", "care", "career", "cargo", "caribou", "carload", "carnation", "carnival", "carol", "carotene", "carp", "carpenter", "carpet", "carpeting", "carport", "carriage", "carrier", "carrot", "carry", "cart", "cartel", "carter", "cartilage", "cartload", "cartoon", "cartridge", "carving", -"cascade", "case", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet", +"cascade", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet", "castle", "casualty", "cat", "catacomb", "catalogue", "catalysis", "catalyst", "catamaran", "catastrophe", "catch", "catcher", "category", "caterpillar", "cathedral", "cation", "catsup", "cattle", "cauliflower", "causal", "cause", "causeway", "caution", "cave", "caviar", "cayenne", "ceiling", "celebration", "celebrity", "celeriac", "celery", "cell", "cellar", "cello", "celsius", "cement", "cemetery", "cenotaph", @@ -184,10 +184,10 @@ std::initializer_list nouns "cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic", "cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", "dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", -"data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", +"data", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", "death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", -"deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", +"deduction", "deed", "deep", "deer", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", "degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den", "denim", "denominator", "density", "dentist", "deodorant", "department", "departure", "dependency", "dependent", "deployment", "deposit", "deposition", "depot", "depression", "depressive", "depth", "deputy", "derby", "derivation", "derivative", "derrick", "descendant", "descent", @@ -250,7 +250,7 @@ std::initializer_list nouns "flytrap", "foal", "foam", "fob", "focus", "fog", "fold", "folder", "folk", "folklore", "follower", "following", "fondue", "font", "food", "foodstuffs", "fool", "foot", "footage", "football", "footnote", "footprint", "footrest", "footstep", "footstool", "footwear", "forage", "forager", "foray", "force", "ford", "forearm", "forebear", "forecast", "forehead", "foreigner", "forelimb", "forest", "forestry", "forever", -"forgery", "fork", "form", "formal", "formamide", "format", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight", +"forgery", "fork", "form", "formal", "formamide", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight", "fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance", "frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight", "freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe", @@ -271,7 +271,7 @@ std::initializer_list nouns "grandfather", "grandma", "grandmom", "grandmother", "grandpa", "grandparent", "grandson", "granny", "granola", "grant", "grape", "grapefruit", "graph", "graphic", "grasp", "grass", "grasshopper", "grassland", "gratitude", "gravel", "gravitas", "gravity", "gravy", "gray", "grease", "greatness", "greed", "green", "greenhouse", "greens", "grenade", "grey", "grid", "grief", -"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole", +"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "grouper", "grouse", "grove", "growth", "grub", "guacamole", "guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty", "guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology", "gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", @@ -332,7 +332,7 @@ std::initializer_list nouns "lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesson", "letter", "lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing", "licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning", -"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line", +"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limitation", "limo", "line", "linen", "liner", "linguist", "linguistics", "lining", "link", "linkage", "linseed", "lion", "lip", "lipid", "lipoprotein", "lipstick", "liquid", "liquidity", "liquor", "list", "listening", "listing", "literate", "literature", "litigation", "litmus", "litter", "littleneck", "liver", "livestock", "living", "lizard", "llama", "load", "loading", "loaf", "loafer", "loan", "lobby", "lobotomy", "lobster", "local", @@ -386,7 +386,7 @@ std::initializer_list nouns "offering", "office", "officer", "official", "offset", "oil", "okra", "oldie", "oleo", "olive", "omega", "omelet", "omission", "omnivore", "oncology", "onion", "online", "onset", "opening", "opera", "operating", "operation", "operator", "ophthalmologist", "opinion", "opium", "opossum", "opponent", "opportunist", "opportunity", "opposite", "opposition", "optimal", "optimisation", "optimist", "optimization", -"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "order", "ordinary", "ordination", "ore", "oregano", "organ", +"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "ordinary", "ordination", "ore", "oregano", "organ", "organisation", "organising", "organization", "organizing", "orient", "orientation", "origin", "original", "originality", "ornament", "osmosis", "osprey", "ostrich", "other", "otter", "ottoman", "ounce", "outback", "outcome", "outfielder", "outfit", "outhouse", "outlaw", "outlay", "outlet", "outline", "outlook", "output", "outrage", "outrigger", "outrun", "outset", "outside", "oval", "ovary", "oven", "overcharge", @@ -532,7 +532,7 @@ std::initializer_list nouns "suspenders", "suspension", "sustainment", "sustenance", "swallow", "swamp", "swan", "swanling", "swath", "sweat", "sweater", "sweatshirt", "sweatshop", "sweatsuit", "sweets", "swell", "swim", "swimming", "swimsuit", "swine", "swing", "switch", "switchboard", "switching", "swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy", -"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop", +"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "tablecloth", "tablet", "tabletop", "tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover", "tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler", @@ -564,7 +564,7 @@ std::initializer_list nouns "tuxedo", "tweet", "tweezers", "twig", "twilight", "twine", "twins", "twist", "twister", "twitter", "type", "typeface", "typewriter", "typhoon", "ukulele", "ultimatum", "umbrella", "unblinking", "uncertainty", "uncle", "underclothes", "underestimate", "underground", "underneath", "underpants", "underpass", "undershirt", "understanding", "understatement", "undertaker", "underwear", "underweight", "underwire", -"underwriting", "unemployment", "unibody", "uniform", "uniformity", "union", "unique", "unit", "unity", "universe", "university", "update", +"underwriting", "unemployment", "unibody", "uniform", "uniformity", "unique", "unit", "unity", "universe", "university", "update", "upgrade", "uplift", "upper", "upstairs", "upward", "urge", "urgency", "urn", "usage", "use", "user", "usher", "usual", "utensil", "utilisation", "utility", "utilization", "vacation", "vaccine", "vacuum", "vagrant", "valance", "valentine", "validate", "validity", "valley", "valuable", "value", "vampire", "van", "vanadyl", "vane", "vanilla", "vanity", "variability", "variable", "variant", "variation", "variety", "vascular", diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference index 0064ac73a09..87d4a303277 100644 --- a/tests/queries/0_stateless/01508_query_obfuscator.reference +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -1,16 +1,16 @@ -SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1')) -SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end +SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Obsidian_id_diverTeam, sweets(Workplace), avgIf(remote('128.0.0.1')) +SELECT treasury_mammoth_hazelnut between nutmeg and span, case when chive >= 116 then switching else null end SELECT - ChimeID, - Testimonial.ID, Testimonial.SipCauseway, - TankfulTRUMPET, - HUMIDITY.TermiteName, HUMIDITY.TermiteSculptural, HUMIDITY.TermiteGuilt, HUMIDITY.TermiteIntensity, HUMIDITY.SipCauseway, HUMIDITY.Coat -FROM merge.tinkle_efficiency + EarthquakeID, + Workout.ID, Workout.CoupleThrill, + MedalEMPIRE, + HOPE.ListingName, HOPE.ListingBomb, HOPE.ListingRamen, HOPE.ListingResult, HOPE.CoupleThrill, HOPE.Smile +FROM merge.marsh_agreement WHERE - FaithSeller >= '2020-10-13' AND FaithSeller <= '2020-10-21' - AND MandolinID = 30750384 - AND intHash32(GafferID) = 448362928 AND intHash64(GafferID) = 12572659331310383983 - AND ChimeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743) - AND Stot = 1 + RecapitulationLeaver >= '2020-10-13' AND RecapitulationLeaver <= '2020-10-21' + AND MasonryID = 30750384 + AND intHash32(EyeballID) = 448362928 AND intHash64(EyeballID) = 12572659331310383983 + AND EarthquakeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743) + AND Photography = 1 From 7e3524caa1cbdf65f686eaaf6d39049734f5ed4a Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 28 Nov 2020 08:17:20 +0000 Subject: [PATCH 14/34] fix spelling errors --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 5 +++-- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 6 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bc9edfac679..b213bb7b6f9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3895,7 +3895,7 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S MergeTreeData::AlterConversions MergeTreeData::getAlterConversionsForPart(const MergeTreeDataPartPtr part) const { - MutationCommands commands = getFirtsAlterMutationCommandsForPart(part); + MutationCommands commands = getFirstAlterMutationCommandsForPart(part); AlterConversions result{}; for (const auto & command : commands) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e0d03cc9c02..1f9ca97abaf 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -905,7 +905,7 @@ protected: /// Used to receive AlterConversions for part and apply them on fly. This /// method has different implementations for replicated and non replicated /// MergeTree because they store mutations in different way. - virtual MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; + virtual MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; /// Moves part to specified space, used in ALTER ... MOVE ... queries bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index df310582803..a1b419ebb89 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -793,7 +793,8 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } -std::shared_ptr StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) +std::shared_ptr StorageMergeTree::selectPartsToMutate( + const StorageMetadataPtr & metadata_snapshot, String * /* disable_reason */, TableLockHolder & /* table_lock_holder */) { std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -1401,7 +1402,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & c } -MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const +MutationCommands StorageMergeTree::getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const { std::lock_guard lock(currently_processing_in_background_mutex); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 86d33e25df0..e0ac17255a2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -231,7 +231,7 @@ protected: std::unique_ptr settings_, bool has_force_restore_data_flag); - MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override; + MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 623562b400e..5857a75131c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6176,7 +6176,7 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const } -MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const +MutationCommands StorageReplicatedMergeTree::getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const { return queue.getFirstAlterMutationCommandsForPart(part); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3cca265a024..90acba9f499 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -582,7 +582,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override; + MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; void startBackgroundMovesIfNeeded() override; From ee38cce1bf6aa29ef18d1940e47153e55e5434ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 28 Nov 2020 15:02:51 +0300 Subject: [PATCH 15/34] Fix toUnixTimestamp(Date()) error (use type name not column type name) Was: Code: 44. DB::Exception: Received from localhost:9000. DB::Exception: Illegal column UInt16 of first argument of function toUnixTimestamp: While processing toUnixTimestamp(today()). Now: Code: 44. DB::Exception: Received from localhost:9000. DB::Exception: Illegal type Date of first argument of function toUnixTimestamp: While processing toUnixTimestamp(today()). --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 968893578f8..1e4bd5c26d6 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -117,7 +117,7 @@ struct ConvertImpl if (std::is_same_v) { if (isDate(named_from.type)) - throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name, + throw Exception("Illegal type " + named_from.type->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } From e635b47299337ba6e79d76ad17ebc58e67f1a01e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 10:43:34 +0300 Subject: [PATCH 16/34] Add a test for StorageJoin and UUID --- .../01594_storage_join_uuid.reference | 1 + .../0_stateless/01594_storage_join_uuid.sql | 29 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/01594_storage_join_uuid.reference create mode 100644 tests/queries/0_stateless/01594_storage_join_uuid.sql diff --git a/tests/queries/0_stateless/01594_storage_join_uuid.reference b/tests/queries/0_stateless/01594_storage_join_uuid.reference new file mode 100644 index 00000000000..44964b11f68 --- /dev/null +++ b/tests/queries/0_stateless/01594_storage_join_uuid.reference @@ -0,0 +1 @@ +00000000-0000-0000-0000-000000000000 diff --git a/tests/queries/0_stateless/01594_storage_join_uuid.sql b/tests/queries/0_stateless/01594_storage_join_uuid.sql new file mode 100644 index 00000000000..9236e9d0cc4 --- /dev/null +++ b/tests/queries/0_stateless/01594_storage_join_uuid.sql @@ -0,0 +1,29 @@ +-- the test from simPod, https://github.com/ClickHouse/ClickHouse/issues/5608 + +DROP TABLE IF EXISTS joint; -- the table name from the original issue. +DROP TABLE IF EXISTS t; + +CREATE TABLE IF NOT EXISTS joint +( + id UUID, + value LowCardinality(String) +) +ENGINE = Join (ANY, LEFT, id); + +CREATE TABLE IF NOT EXISTS t +( + id UUID, + d DateTime +) +ENGINE = MergeTree +PARTITION BY toDate(d) +ORDER BY id; + +insert into joint VALUES ('00000000-0000-0000-0000-000000000000', 'yo'); +insert into t VALUES ('00000000-0000-0000-0000-000000000000', now()); + +SELECT id FROM t +ANY LEFT JOIN joint ON t.id = joint.id; + +DROP TABLE joint; +DROP TABLE t; From 27063ad160ed9926274bab2bb1b38cd83af169ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:10:39 +0300 Subject: [PATCH 17/34] Update Poco: Fix thread counter leak --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index f49c6ab8d3a..f3d791f6568 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit f49c6ab8d3aa71828bd1b411485c21722e8c9d82 +Subproject commit f3d791f6568b99366d089b4479f76a515beb66d5 From c679d8b357d9c2dd3872d16c97fd9bac4f9a8f0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:42:17 +0300 Subject: [PATCH 18/34] Update Cassandra for BoringSSL --- contrib/cassandra | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cassandra b/contrib/cassandra index a49b4e0e269..d10187efb25 160000 --- a/contrib/cassandra +++ b/contrib/cassandra @@ -1 +1 @@ -Subproject commit a49b4e0e2696a4b8ef286a5b9538d1cbe8490509 +Subproject commit d10187efb25b26da391def077edf3c6f2f3a23dd From 1f4087b8d0bdb019fa90a1587f91a3ac4516335f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:48:43 +0300 Subject: [PATCH 19/34] Update Kafka for BoringSSL --- contrib/librdkafka | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 2090cbf56b7..9902bc4fb18 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2090cbf56b715247ec2be7f768707a7ab1bf7ede +Subproject commit 9902bc4fb18bb441fa55ca154b341cdda191e5d3 From 6f660f49035431b504f36f0b148c70b9c7fa56a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:49:46 +0300 Subject: [PATCH 20/34] Update MariaDB for BoringSSL --- contrib/mariadb-connector-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index 1485b0de3ea..e05523ca7c1 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit 1485b0de3eaa1508dfe49a5ba1e4aa2a71fd8335 +Subproject commit e05523ca7c1fb8d095b612a1b1cfe96e199ffb17 From 4d4aa931652302671a181905e3b009b5267de740 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:51:32 +0300 Subject: [PATCH 21/34] Update OpenLDAP for BoringSSL --- .gitmodules | 2 +- contrib/openldap | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 6db83baa056..bd06d9d9acc 100644 --- a/.gitmodules +++ b/.gitmodules @@ -157,7 +157,7 @@ url = https://github.com/ClickHouse-Extras/libcpuid.git [submodule "contrib/openldap"] path = contrib/openldap - url = https://github.com/openldap/openldap.git + url = https://github.com/ClickHouse-Extras/openldap.git [submodule "contrib/AMQP-CPP"] path = contrib/AMQP-CPP url = https://github.com/ClickHouse-Extras/AMQP-CPP.git diff --git a/contrib/openldap b/contrib/openldap index 34b9ba94b30..0208811b604 160000 --- a/contrib/openldap +++ b/contrib/openldap @@ -1 +1 @@ -Subproject commit 34b9ba94b30319ed6389a4e001d057f7983fe363 +Subproject commit 0208811b6043ca06fda8631a5e473df1ec515ccb From a96d9104d98f0f8e30f343d4a48a081b66f97115 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 11:53:15 +0300 Subject: [PATCH 22/34] Update AMQP for BoringSSL --- contrib/AMQP-CPP | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/AMQP-CPP b/contrib/AMQP-CPP index d63e1f01658..03781aaff0f 160000 --- a/contrib/AMQP-CPP +++ b/contrib/AMQP-CPP @@ -1 +1 @@ -Subproject commit d63e1f016582e9faaaf279aa24513087a07bc6e7 +Subproject commit 03781aaff0f10ef41f902b8cf865fe0067180c10 From 3bd9a7d3bd723bb0c589c1fdc616f0efffec7297 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 15:23:16 +0300 Subject: [PATCH 23/34] Add a test for #12297 --- .../0_stateless/01596_full_join_chertus.reference | 6 ++++++ tests/queries/0_stateless/01596_full_join_chertus.sql | 9 +++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/01596_full_join_chertus.reference create mode 100644 tests/queries/0_stateless/01596_full_join_chertus.sql diff --git a/tests/queries/0_stateless/01596_full_join_chertus.reference b/tests/queries/0_stateless/01596_full_join_chertus.reference new file mode 100644 index 00000000000..5b1f6aa945d --- /dev/null +++ b/tests/queries/0_stateless/01596_full_join_chertus.reference @@ -0,0 +1,6 @@ +UInt64 LowCardinality(UInt64) LowCardinality(String) String +UInt64 LowCardinality(UInt64) LowCardinality(String) String +UInt64 LowCardinality(UInt64) LowCardinality(String) String +UInt64 LowCardinality(UInt64) LowCardinality(String) String +UInt64 LowCardinality(UInt64) LowCardinality(String) String +UInt64 LowCardinality(UInt64) LowCardinality(String) String diff --git a/tests/queries/0_stateless/01596_full_join_chertus.sql b/tests/queries/0_stateless/01596_full_join_chertus.sql new file mode 100644 index 00000000000..162b9f7be8f --- /dev/null +++ b/tests/queries/0_stateless/01596_full_join_chertus.sql @@ -0,0 +1,9 @@ +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +full join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select toTypeName(js1.k), toTypeName(js2.k), toTypeName(js1.s), toTypeName(js2.s) +from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +full join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 +using k order by js1.k, js2.k; From 3ccc4d66a166edb8db7d30b8208649609e95edd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 16:02:35 +0300 Subject: [PATCH 24/34] Add a test for #9490 --- .../0_stateless/01597_columns_list_ignored.reference | 1 + tests/queries/0_stateless/01597_columns_list_ignored.sh | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01597_columns_list_ignored.reference create mode 100755 tests/queries/0_stateless/01597_columns_list_ignored.sh diff --git a/tests/queries/0_stateless/01597_columns_list_ignored.reference b/tests/queries/0_stateless/01597_columns_list_ignored.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01597_columns_list_ignored.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01597_columns_list_ignored.sh b/tests/queries/0_stateless/01597_columns_list_ignored.sh new file mode 100755 index 00000000000..3098eb5161a --- /dev/null +++ b/tests/queries/0_stateless/01597_columns_list_ignored.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --query "describe table file('', TSV, 'a int, b.c int')" 2>&1 | grep -F -c 'Syntax error' From 5edda3c6bd5b154894cdde6879042808b813aa79 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 16:04:57 +0300 Subject: [PATCH 25/34] Add a test for #11803 --- tests/queries/0_stateless/01598_memory_limit_zeros.reference | 0 tests/queries/0_stateless/01598_memory_limit_zeros.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01598_memory_limit_zeros.reference create mode 100644 tests/queries/0_stateless/01598_memory_limit_zeros.sql diff --git a/tests/queries/0_stateless/01598_memory_limit_zeros.reference b/tests/queries/0_stateless/01598_memory_limit_zeros.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01598_memory_limit_zeros.sql b/tests/queries/0_stateless/01598_memory_limit_zeros.sql new file mode 100644 index 00000000000..e90d7bbccb7 --- /dev/null +++ b/tests/queries/0_stateless/01598_memory_limit_zeros.sql @@ -0,0 +1,2 @@ +SET max_memory_usage = 1; +select 'test', count(*) from zeros_mt(1000000) where not ignore(zero); -- { serverError 241 } From f9b1e77808a887fadefb2e92504f5b68adad6e58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 16:30:16 +0300 Subject: [PATCH 26/34] ParserAlterQuery: fix strange code --- src/Parsers/ParserAlterQuery.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 7050614007e..f0d28a857f3 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -588,7 +588,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else return false; - } if (command->col_decl) @@ -601,6 +600,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->order_by); if (command->sample_by) command->children.push_back(command->sample_by); + if (command->index_decl) + command->children.push_back(command->index_decl); + if (command->index) + command->children.push_back(command->index); + if (command->constraint_decl) + command->children.push_back(command->constraint_decl); + if (command->constraint) + command->children.push_back(command->constraint); if (command->predicate) command->children.push_back(command->predicate); if (command->update_assignments) @@ -613,6 +620,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->ttl); if (command->settings_changes) command->children.push_back(command->settings_changes); + if (command->select) + command->children.push_back(command->select); + if (command->rename_to) + command->children.push_back(command->rename_to); return true; } From 4f47545bc856e0e0db704bf032b85d63a6c330f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 16:42:42 +0300 Subject: [PATCH 27/34] Add a test for #10976 --- .../01599_mutation_query_params.reference | 2 ++ .../01599_mutation_query_params.sh | 31 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/01599_mutation_query_params.reference create mode 100755 tests/queries/0_stateless/01599_mutation_query_params.sh diff --git a/tests/queries/0_stateless/01599_mutation_query_params.reference b/tests/queries/0_stateless/01599_mutation_query_params.reference new file mode 100644 index 00000000000..8e6c69b87b2 --- /dev/null +++ b/tests/queries/0_stateless/01599_mutation_query_params.reference @@ -0,0 +1,2 @@ +2 1 +1 1 diff --git a/tests/queries/0_stateless/01599_mutation_query_params.sh b/tests/queries/0_stateless/01599_mutation_query_params.sh new file mode 100755 index 00000000000..46e11b0865f --- /dev/null +++ b/tests/queries/0_stateless/01599_mutation_query_params.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -n --query " +DROP TABLE IF EXISTS test; + +CREATE TABLE test +( + id UUID, + date_time DateTime, + x UInt32, + y UInt32 +) ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(date_time) +ORDER BY (date_time); + +INSERT INTO test (x, y) VALUES (2, 1); +" + +$CLICKHOUSE_CLIENT --query "SELECT x, y FROM test" + +$CLICKHOUSE_CLIENT --mutations_sync 1 --param_x 1 --param_y 1 --query " +ALTER TABLE test +UPDATE x = {x:UInt32} +WHERE y = {y:UInt32}; +" + +$CLICKHOUSE_CLIENT --query "SELECT x, y FROM test" From 4a66db1bf4afc8c9a847bb0d2449200495cee90c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 16:43:24 +0300 Subject: [PATCH 28/34] Fix #10976 --- src/Parsers/ASTAssignment.h | 17 +++++++---------- src/Parsers/ParserAlterQuery.cpp | 7 ++++--- src/Storages/MutationCommands.cpp | 2 +- 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Parsers/ASTAssignment.h b/src/Parsers/ASTAssignment.h index 6753711f9e3..88d4bb96c15 100644 --- a/src/Parsers/ASTAssignment.h +++ b/src/Parsers/ASTAssignment.h @@ -10,21 +10,18 @@ class ASTAssignment : public IAST { public: String column_name; - ASTPtr expression; + + ASTPtr expression() const + { + return children.at(0); + } String getID(char delim) const override { return "Assignment" + (delim + column_name); } ASTPtr clone() const override { auto res = std::make_shared(*this); - res->children.clear(); - - if (expression) - { - res->expression = expression->clone(); - res->children.push_back(res->expression); - } - + res->children = { expression()->clone() }; return res; } @@ -37,7 +34,7 @@ protected: settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : ""); - expression->formatImpl(settings, state, frame); + expression()->formatImpl(settings, state, frame); } }; diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index f0d28a857f3..b3f40eb42eb 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -667,12 +667,13 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!s_equals.ignore(pos, expected)) return false; - if (!p_expression.parse(pos, assignment->expression, expected)) + ASTPtr expression; + if (!p_expression.parse(pos, expression, expected)) return false; tryGetIdentifierNameInto(column, assignment->column_name); - if (assignment->expression) - assignment->children.push_back(assignment->expression); + if (expression) + assignment->children.push_back(expression); return true; } diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index daa7665403f..f832e674444 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -47,7 +47,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, for (const ASTPtr & assignment_ast : command->update_assignments->children) { const auto & assignment = assignment_ast->as(); - auto insertion = res.column_to_update_expression.emplace(assignment.column_name, assignment.expression); + auto insertion = res.column_to_update_expression.emplace(assignment.column_name, assignment.expression()); if (!insertion.second) throw Exception("Multiple assignments in the single statement to column " + backQuote(assignment.column_name), ErrorCodes::MULTIPLE_ASSIGNMENTS_TO_COLUMN); From a965dba0b90c76cfbd44a517144a03acfc943a11 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 29 Nov 2020 16:45:08 +0300 Subject: [PATCH 29/34] Update 01599_mutation_query_params.sh --- tests/queries/0_stateless/01599_mutation_query_params.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01599_mutation_query_params.sh b/tests/queries/0_stateless/01599_mutation_query_params.sh index 46e11b0865f..6bc12b5409e 100755 --- a/tests/queries/0_stateless/01599_mutation_query_params.sh +++ b/tests/queries/0_stateless/01599_mutation_query_params.sh @@ -29,3 +29,4 @@ WHERE y = {y:UInt32}; " $CLICKHOUSE_CLIENT --query "SELECT x, y FROM test" +$CLICKHOUSE_CLIENT --query "DROP TABLE test" From f80ae936ad5e83839e2104a345c16d58148febe0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 17:50:42 +0300 Subject: [PATCH 30/34] Fix parsing of multiline queries in interactive mode #13654 --- base/common/LineReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/LineReader.cpp b/base/common/LineReader.cpp index b2bc929a1df..a32906dd5a5 100644 --- a/base/common/LineReader.cpp +++ b/base/common/LineReader.cpp @@ -127,7 +127,7 @@ String LineReader::readLine(const String & first_prompt, const String & second_p } #endif - line += (line.empty() ? "" : " ") + input; + line += (line.empty() ? "" : "\n") + input; if (!need_next_line) break; From 98f9670f6915b347dd1e73ab00e19a452110ccf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 17:50:50 +0300 Subject: [PATCH 31/34] Add a test --- ...ne_input_and_singleline_comments.reference | 0 ...multiline_input_and_singleline_comments.sh | 25 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.reference create mode 100755 tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.sh diff --git a/tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.reference b/tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.sh b/tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.sh new file mode 100755 index 00000000000..5b7d1356dc6 --- /dev/null +++ b/tests/queries/0_stateless/01599_multiline_input_and_singleline_comments.sh @@ -0,0 +1,25 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 5 +match_max 100000 + +if ![info exists env(CLICKHOUSE_PORT_TCP)] {set env(CLICKHOUSE_PORT_TCP) 9000} + +spawn clickhouse-client --multiline --port "$env(CLICKHOUSE_PORT_TCP)" +expect ":) " + +# Make a query +send -- "SELECT 1\r" +expect ":-] " +send -- "-- xxx\r" +expect ":-] " +send -- ", 2\r" +expect ":-] " +send -- ";\r" + +expect "│ 1 │ 2 │" +expect ":) " + +send -- "\4" +expect eof From 7f1a8b15fd3a7a01a84ce48a98a6bed1823c26ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Nov 2020 21:21:46 +0300 Subject: [PATCH 32/34] Fix bad test --- .../0_stateless/01317_no_password_in_command_line.sh | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.sh b/tests/queries/0_stateless/01317_no_password_in_command_line.sh index 0f6d8117307..07ac2b88832 100755 --- a/tests/queries/0_stateless/01317_no_password_in_command_line.sh +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.sh @@ -22,7 +22,7 @@ do break fi - if ! kill -0 -- $bg_query + if ! kill -0 -- $bg_query 2>/dev/null then # The SELECT sleep(1) query finished earlier that we could grep for it in the process list, but it should have run for at least one second. It is Ok. break @@ -30,8 +30,6 @@ do done ps auxw | grep -F -- '--password' | grep -F hello ||: -# Check that it is still running -kill -0 -- $bg_query wait # Once again with different syntax @@ -46,7 +44,7 @@ do break fi - if ! kill -0 -- $bg_query + if ! kill -0 -- $bg_query 2>/dev/null then # The SELECT sleep(1) query finished earlier that we could grep for it in the process list, but it should have run for at least one second. It is Ok. break @@ -54,8 +52,6 @@ do done ps auxw | grep -F -- '--password' | grep -F hello ||: -# Check that it is still running -kill -0 -- $bg_query wait $CLICKHOUSE_CLIENT --query "DROP USER user" From 104c1a72e823bef0c2e2ef5b7f6c9786970a268e Mon Sep 17 00:00:00 2001 From: AnaUvarova <64017504+AnaUvarova@users.noreply.github.com> Date: Sun, 29 Nov 2020 22:57:51 +0300 Subject: [PATCH 33/34] DOCSUP-2956 system events show zero values setting (#16794) * Draft * Material * Dratf is ready * + * Update docs/en/operations/settings/settings.md Co-authored-by: BayoNet * Comments fixed * Russian translation * Final edits * Update docs/ru/operations/settings/settings.md Co-authored-by: BayoNet * Update docs/en/operations/settings/settings.md * Update docs/ru/operations/settings/settings.md * one more adopters * Update docs/en/introduction/adopters.md * Update adopters.md Fixed the sorting order. Co-authored-by: BayoNet --- docs/en/introduction/adopters.md | 1 + docs/en/operations/settings/settings.md | 41 ++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 42 +++++++++++++++++++++++++ 3 files changed, 84 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 6cb65d5b9b9..10fd173f76f 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -23,6 +23,7 @@ toc_title: Adopters | BIGO | Video | Computing Platform | — | — | [Blog Article, August 2020](https://www.programmersought.com/article/44544895251/) | | Bloomberg | Finance, Media | Monitoring | 102 servers | — | [Slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | | Bloxy | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | +| Bytedance | Social platforms | — | — | — | [The ClickHouse Meetup East, October 2020](https://www.youtube.com/watch?v=ckChUkC3Pns) | | CardsMobile | Finance | Analytics | — | — | [VC.ru](https://vc.ru/s/cardsmobile/143449-rukovoditel-gruppy-analiza-dannyh) | | CARTO | Business Intelligence | Geo analytics | — | — | [Geospatial processing with ClickHouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) | | CERN | Research | Experiment | — | — | [Press release, April 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) | diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ba899754b18..8346d5ceac9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2293,6 +2293,47 @@ Result: └─────────────────────────┴─────────┘ ``` +## system_events_show_zero_values {#system_events_show_zero_values} + +Allows to select zero-valued events from [`system.events`](../../operations/system-tables/events.md). + +Some monitoring systems require passing all the metrics values to them for each checkpoint, even if the metric value is zero. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. + +**Examples** + +Query + +```sql +SELECT * FROM system.events WHERE event='QueryMemoryLimitExceeded'; +``` + +Result + +```text +Ok. +``` + +Query +```sql +SET system_events_show_zero_values = 1; +SELECT * FROM system.events WHERE event='QueryMemoryLimitExceeded'; +``` + +Result + +```text +┌─event────────────────────┬─value─┬─description───────────────────────────────────────────┐ +│ QueryMemoryLimitExceeded │ 0 │ Number of times when memory limit exceeded for query. │ +└──────────────────────────┴───────┴───────────────────────────────────────────────────────┘ +``` + ## allow_experimental_bigint_types {#allow_experimental_bigint_types} Enables or disables integer values exceeding the range that is supported by the int data type. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index af0fc3e6137..b04a927f944 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2099,6 +2099,48 @@ SELECT TOP 3 name, value FROM system.settings; └─────────────────────────┴─────────┘ ``` +## system_events_show_zero_values {#system_events_show_zero_values} + +Позволяет выбрать события с нулевыми значениями из таблицы [`system.events`](../../operations/system-tables/events.md). + +В некоторые системы мониторинга вам нужно передать значения всех измерений (для каждой контрольной точки), даже если в результате — "0". + +Возможные значения: + +- 0 — настройка отключена — вы получите все события. +- 1 — настройка включена — вы сможете отсортировать события по нулевым и остальным значениям. + +Значение по умолчанию: `0`. + +**Примеры** + +Запрос + +```sql +SELECT * FROM system.events WHERE event='QueryMemoryLimitExceeded'; +``` + +Результат + +```text +Ok. +``` + +Запрос + +```sql +SET system_events_show_zero_values = 1; +SELECT * FROM system.events WHERE event='QueryMemoryLimitExceeded'; +``` + +Результат + +```text +┌─event────────────────────┬─value─┬─description───────────────────────────────────────────┐ +│ QueryMemoryLimitExceeded │ 0 │ Number of times when memory limit exceeded for query. │ +└──────────────────────────┴───────┴───────────────────────────────────────────────────────┘ +``` + ## allow_experimental_bigint_types {#allow_experimental_bigint_types} Включает или отключает поддержку целочисленных значений, превышающих максимальное значение, допустимое для типа `int`. From 5b3023c7b60c2dc63f4f5e10fda1556e9b4209b2 Mon Sep 17 00:00:00 2001 From: Robert Hodges Date: Sun, 29 Nov 2020 12:22:00 -0800 Subject: [PATCH 34/34] Update README.md Add listings for upcoming SF Bay Area ClickHouse meetups. --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index fc39cf90861..393deb9b86f 100644 --- a/README.md +++ b/README.md @@ -14,3 +14,7 @@ ClickHouse® is an open-source column-oriented database management system that a * [Yandex.Messenger channel](https://yandex.ru/chat/#/join/20e380d9-c7be-4123-ab06-e95fb946975e) shares announcements and useful links in Russian. * [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. + +## Upcoming Events +* [SF Bay Area ClickHouse December Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/274498897/) on 2 December 2020. +* [SF Bay Area ClickHouse January Virtual Office Hours (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/274273549/) on 20 January 2020.