From 8b07a7f1807ef771b7b163b7728db215f9c7552a Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 27 Jul 2021 15:35:20 +0100 Subject: [PATCH 01/64] Store exception generated when we tried to update the queue last time The use case is to alert when queue contains broken entries. Especially important when ClickHouse breaks backwards compatibility between versions and log entries written by newer versions aren't parseable by old versions. ``` Code: 27, e.displayText() = DB::Exception: Cannot parse input: expected 'quorum: ' before: 'merge_type: 2\n' ``` --- .../ReplicatedMergeTreeRestartingThread.cpp | 17 +++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 31 ++++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 2 ++ src/Storages/System/StorageSystemReplicas.cpp | 2 ++ 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 1c9921aad1d..eadd414f1d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -153,11 +153,20 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() storage.cloneReplicaIfNeeded(zookeeper); - storage.queue.load(zookeeper); + try + { + storage.queue.load(zookeeper); + + /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); + /// because cleanup_thread doesn't delete log_pointer of active replicas. + storage.queue.pullLogsToQueue(zookeeper); + } + catch (...) + { + storage.last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); + throw; + } - /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); - /// because cleanup_thread doesn't delete log_pointer of active replicas. - storage.queue.pullLogsToQueue(zookeeper); storage.queue.removeCurrentPartsFromMutations(); storage.last_queue_update_finish_time.store(time(nullptr)); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d44b86fe9bb..8966a34e825 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3087,21 +3087,24 @@ void StorageReplicatedMergeTree::queueUpdatingTask() last_queue_update_finish_time.store(time(nullptr)); queue_update_in_progress = false; } - catch (const Coordination::Exception & e) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - - if (e.code == Coordination::Error::ZSESSIONEXPIRED) - { - restarting_thread.wakeup(); - return; - } - - queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); - } catch (...) { + last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); tryLogCurrentException(log, __PRETTY_FUNCTION__); + + try + { + throw; + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::Error::ZSESSIONEXPIRED) + { + restarting_thread.wakeup(); + return; + } + } + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); } } @@ -5562,6 +5565,10 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.total_replicas = 0; res.active_replicas = 0; + MultiVersion::Version queue_exception = last_queue_update_exception.get(); + if (queue_exception) + res.last_queue_update_exception = *queue_exception; + if (with_zk_fields && !res.is_session_expired) { try diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 800f419cb76..9c3b9b12e37 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -174,6 +174,7 @@ public: UInt64 absolute_delay; UInt8 total_replicas; UInt8 active_replicas; + String last_queue_update_exception; /// If the error has happened fetching the info from ZooKeeper, this field will be set. String zookeeper_exception; }; @@ -329,6 +330,7 @@ private: ReplicatedMergeTreeQueue queue; std::atomic last_queue_update_start_time{0}; std::atomic last_queue_update_finish_time{0}; + MultiVersion last_queue_update_exception; DataPartsExchange::Fetcher fetcher; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index fc33c6b421b..3af7352616f 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -50,6 +50,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) { "absolute_delay", std::make_shared() }, { "total_replicas", std::make_shared() }, { "active_replicas", std::make_shared() }, + { "last_queue_update_exception", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, })); setInMemoryMetadata(storage_metadata); @@ -183,6 +184,7 @@ Pipe StorageSystemReplicas::read( res_columns[col_num++]->insert(status.absolute_delay); res_columns[col_num++]->insert(status.total_replicas); res_columns[col_num++]->insert(status.active_replicas); + res_columns[col_num++]->insert(status.last_queue_update_exception); res_columns[col_num++]->insert(status.zookeeper_exception); } From 3f291b024a315e6afa3401ebbc5b52fb49e0e3be Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 9 Aug 2021 13:58:23 +0100 Subject: [PATCH 02/64] Use plain mutex instead of MultiVersion --- .../ReplicatedMergeTreeRestartingThread.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 39 +++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 5 ++- 3 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index eadd414f1d5..edd0876c6e9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -163,7 +163,8 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() } catch (...) { - storage.last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); + std::unique_lock lock(storage.last_queue_update_exception_lock); + storage.last_queue_update_exception = getCurrentExceptionMessage(false); throw; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8966a34e825..a8b6d4170d9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3073,6 +3073,12 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke zookeeper->set(fs::path(replica_path) / "is_lost", "0"); } +String StorageReplicatedMergeTree::getLastQueueUpdateException() const +{ + std::unique_lock lock(last_queue_update_exception_lock); + return last_queue_update_exception; +} + void StorageReplicatedMergeTree::queueUpdatingTask() { @@ -3087,24 +3093,28 @@ void StorageReplicatedMergeTree::queueUpdatingTask() last_queue_update_finish_time.store(time(nullptr)); queue_update_in_progress = false; } - catch (...) + catch (const Coordination::Exception & e) { - last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); tryLogCurrentException(log, __PRETTY_FUNCTION__); - try + std::unique_lock lock(last_queue_update_exception_lock); + last_queue_update_exception = getCurrentExceptionMessage(false); + + if (e.code == Coordination::Error::ZSESSIONEXPIRED) { - throw; - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::Error::ZSESSIONEXPIRED) - { - restarting_thread.wakeup(); - return; - } + restarting_thread.wakeup(); + return; } + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + std::unique_lock lock(last_queue_update_exception_lock); + last_queue_update_exception = getCurrentExceptionMessage(false); + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); } } @@ -5564,10 +5574,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.log_pointer = 0; res.total_replicas = 0; res.active_replicas = 0; - - MultiVersion::Version queue_exception = last_queue_update_exception.get(); - if (queue_exception) - res.last_queue_update_exception = *queue_exception; + res.last_queue_update_exception = getLastQueueUpdateException(); if (with_zk_fields && !res.is_session_expired) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9c3b9b12e37..4741d8b4605 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -330,7 +330,10 @@ private: ReplicatedMergeTreeQueue queue; std::atomic last_queue_update_start_time{0}; std::atomic last_queue_update_finish_time{0}; - MultiVersion last_queue_update_exception; + + mutable std::mutex last_queue_update_exception_lock; + String last_queue_update_exception; + String getLastQueueUpdateException() const; DataPartsExchange::Fetcher fetcher; From 554231c6fc0d30017ab72f222e6d4a0248e29b88 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Aug 2021 17:16:44 +0300 Subject: [PATCH 03/64] Support positional arguments for GROUP BY, ORDER BY, LIMIT BY --- src/Core/Settings.h | 1 + src/Interpreters/ExpressionAnalyzer.cpp | 61 +++++++++++- src/Interpreters/TreeOptimizer.cpp | 8 +- .../02006_test_positional_arguments.reference | 97 +++++++++++++++++++ .../02006_test_positional_arguments.sql | 30 ++++++ 5 files changed, 193 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02006_test_positional_arguments.reference create mode 100644 tests/queries/0_stateless/02006_test_positional_arguments.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e1bd1d29153..90e700c8906 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,6 +114,7 @@ class IColumn; M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ + M(Bool, allow_group_by_column_number, true, "Allow to perform positional group by: GROUP BY {column number}.", 0) \ \ M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "", 0) \ diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 77598e69c00..cabddcac15c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -238,13 +238,37 @@ void ExpressionAnalyzer::analyzeAggregation() { NameSet unique_keys; ASTs & group_asts = select_query->groupBy()->children; + const auto & columns = syntax->source_columns; + for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) { ssize_t size = group_asts.size(); getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); + if (getContext()->getSettingsRef().enable_positional_arguments) + { + /// Case when GROUP BY element is position. + /// Do not consider case when GROUP BY element is expression, even if all values are contants. + /// (because does it worth it to first check that exactly all elements in expression are positions + /// and then traverse once again to make replacement?) + if (const auto * ast_literal = typeid_cast(group_asts[i].get())) + { + auto which = ast_literal->value.getType(); + if (which == Field::Types::UInt64) + { + auto pos = ast_literal->value.get(); + if ((0 < pos) && (pos <= columns.size())) + { + const auto & column_name = std::next(columns.begin(), pos - 1)->name; + group_asts[i] = std::make_shared(column_name); + } + } + } + } + const auto & column_name = group_asts[i]->getColumnName(); const auto * node = temp_actions->tryFindInIndex(column_name); + if (!node) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); @@ -1228,7 +1252,24 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai const auto * ast = child->as(); if (!ast || ast->children.empty()) throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + ASTPtr order_expression = ast->children.at(0); + const auto & columns = syntax->source_columns; + + if (auto * ast_literal = typeid_cast(order_expression.get())) + { + auto which = ast_literal->value.getType(); + if (which == Field::Types::UInt64) + { + auto pos = ast_literal->value.get(); + if ((0 < pos) && (pos <= columns.size())) + { + const auto & column_name = std::next(columns.begin(), pos - 1)->name; + child->children[0] = std::make_shared(column_name); + } + } + } + step.addRequiredOutput(order_expression->getColumnName()); if (ast->with_fill) @@ -1277,9 +1318,25 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain aggregated_names.insert(column.name); } - for (const auto & child : select_query->limitBy()->children) + auto & children = select_query->limitBy()->children; + for (size_t i = 0; i < children.size(); ++i) { - auto child_name = child->getColumnName(); + const auto & columns = syntax->source_columns; + if (auto * ast_literal = typeid_cast(children[i].get())) + { + auto which = ast_literal->value.getType(); + if (which == Field::Types::UInt64) + { + auto pos = ast_literal->value.get(); + if ((0 < pos) && (pos <= columns.size())) + { + const auto & column_name = std::next(columns.begin(), pos - 1)->name; + children[i] = std::make_shared(column_name); + } + } + } + + auto child_name = children[i]->getColumnName(); if (!aggregated_names.count(child_name)) step.addRequiredOutput(std::move(child_name)); } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c1a265d9a06..8257e54defc 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -69,7 +69,9 @@ const std::unordered_set possibly_injective_function_names void appendUnusedGroupByColumn(ASTSelectQuery * select_query, const NameSet & source_columns) { /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. - UInt64 unused_column = 0; + /// Also start unused_column integer from source_columns.size() + 1, because lower numbers ([1, source_columns.size()]) + /// might be in positional GROUP BY. + UInt64 unused_column = source_columns.size() + 1; String unused_column_name = toString(unused_column); while (source_columns.count(unused_column_name)) @@ -111,6 +113,8 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum group_exprs.pop_back(); }; + const auto & settings = context->getSettingsRef(); + /// iterate over each GROUP BY expression, eliminate injective function calls and literals for (size_t i = 0; i < group_exprs.size();) { @@ -164,7 +168,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum std::back_inserter(group_exprs), is_literal ); } - else if (is_literal(group_exprs[i])) + else if (is_literal(group_exprs[i]) && !settings.enable_positional_arguments) { remove_expr_at_index(i); } diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference new file mode 100644 index 00000000000..ed6fdb53eb3 --- /dev/null +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -0,0 +1,97 @@ +-- { echo } +select * from test order by 1; +0 1 4 +0 0 2 +0 0 1 +0 1 2 +0 1 2 +0 0 2 +0 1 1 +0 0 3 +0 0 4 +0 1 3 +1 2 4 +1 1 1 +1 2 2 +1 1 2 +1 2 2 +1 1 2 +1 2 1 +1 1 3 +1 1 4 +1 2 3 +select * from test order by 3; +0 0 1 +1 1 1 +0 1 1 +1 2 1 +0 1 2 +0 0 2 +1 2 2 +1 1 2 +0 0 2 +1 1 2 +0 1 2 +1 2 2 +0 1 3 +1 2 3 +0 0 3 +1 1 3 +0 1 4 +1 2 4 +0 0 4 +1 1 4 +select col1, col2 from test group by col1, col2 order by col2; +0 0 +0 1 +1 1 +1 2 +select col1, col2 from test group by 1, 2 order by 2; +0 0 +0 1 +1 1 +1 2 +select col2, col3 from test group by col3, col2 order by col3; +0 1 +2 1 +1 1 +0 2 +1 2 +2 2 +0 3 +2 3 +1 3 +0 4 +2 4 +1 4 +select col2, col3 from test group by 3, 2 order by 3; +0 1 +2 1 +1 1 +0 2 +1 2 +2 2 +0 3 +2 3 +1 3 +0 4 +2 4 +1 4 +select col2 from test group by 2 order by 2; +0 +1 +2 +select col2 + 100 from test group by 2 order by col2; +100 +101 +102 +select * from test order by col3 limit 1 by col3; +0 0 1 +0 1 2 +0 0 3 +0 0 4 +select * from test order by 3 limit 1 by 3; +0 0 1 +0 1 2 +0 0 3 +0 1 4 diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql new file mode 100644 index 00000000000..61867c1b564 --- /dev/null +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -0,0 +1,30 @@ +drop table if exists test; +create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); +insert into test select number, number, 1 from numbers(2); +insert into test select number, number, 2 from numbers(2); +insert into test select number, number+1, 1 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); +insert into test select number, number, 3 from numbers(2); +insert into test select number, number, 4 from numbers(2); +insert into test select number, number+1, 3 from numbers(2); +insert into test select number, number+1, 4 from numbers(2); +insert into test select number, number, 2 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); + +set enable_positional_arguments = 1; + +-- { echo } +select * from test order by 1; +select * from test order by 3; + +select col1, col2 from test group by col1, col2 order by col2; +select col1, col2 from test group by 1, 2 order by 2; + +select col2, col3 from test group by col3, col2 order by col3; +select col2, col3 from test group by 3, 2 order by 3; + +select col2 from test group by 2 order by 2; +select col2 + 100 from test group by 2 order by col2; + +select * from test order by col3 limit 1 by col3; +select * from test order by 3 limit 1 by 3; From 503a5edc591a5d66e6a0e70ef27730c1a496d410 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Aug 2021 13:57:37 +0000 Subject: [PATCH 04/64] Fix --- src/Core/Settings.h | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 80 ++++++++--------- .../02006_test_positional_arguments.reference | 87 ++++++++----------- .../02006_test_positional_arguments.sql | 46 +++++----- 4 files changed, 96 insertions(+), 119 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 90e700c8906..bddd57c45dd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,7 +114,7 @@ class IColumn; M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ - M(Bool, allow_group_by_column_number, true, "Allow to perform positional group by: GROUP BY {column number}.", 0) \ + M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ \ M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "", 0) \ diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cabddcac15c..42699db8b8e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -162,6 +162,25 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(); } +static ASTPtr checkPositionalArgument(ASTPtr argument, const NamesAndTypesList & columns) +{ + /// Case when GROUP BY element is position. + /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are contants. + if (auto * ast_literal = typeid_cast(argument.get())) + { + auto which = ast_literal->value.getType(); + if (which == Field::Types::UInt64) + { + auto pos = ast_literal->value.get(); + if ((0 < pos) && (pos <= columns.size())) + { + const auto & column_name = std::next(columns.begin(), pos - 1)->name; + return std::make_shared(column_name); + } + } + } + return nullptr; +} void ExpressionAnalyzer::analyzeAggregation() { @@ -247,23 +266,9 @@ void ExpressionAnalyzer::analyzeAggregation() if (getContext()->getSettingsRef().enable_positional_arguments) { - /// Case when GROUP BY element is position. - /// Do not consider case when GROUP BY element is expression, even if all values are contants. - /// (because does it worth it to first check that exactly all elements in expression are positions - /// and then traverse once again to make replacement?) - if (const auto * ast_literal = typeid_cast(group_asts[i].get())) - { - auto which = ast_literal->value.getType(); - if (which == Field::Types::UInt64) - { - auto pos = ast_literal->value.get(); - if ((0 < pos) && (pos <= columns.size())) - { - const auto & column_name = std::next(columns.begin(), pos - 1)->name; - group_asts[i] = std::make_shared(column_name); - } - } - } + auto new_argument = checkPositionalArgument(group_asts[i], columns); + if (new_argument) + group_asts[i] = new_argument; } const auto & column_name = group_asts[i]->getColumnName(); @@ -1247,29 +1252,22 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai bool with_fill = false; NameSet order_by_keys; + const auto & columns = syntax->source_columns; + for (auto & child : select_query->orderBy()->children) { - const auto * ast = child->as(); + auto * ast = child->as(); if (!ast || ast->children.empty()) throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); - ASTPtr order_expression = ast->children.at(0); - const auto & columns = syntax->source_columns; - - if (auto * ast_literal = typeid_cast(order_expression.get())) + if (getContext()->getSettingsRef().enable_positional_arguments) { - auto which = ast_literal->value.getType(); - if (which == Field::Types::UInt64) - { - auto pos = ast_literal->value.get(); - if ((0 < pos) && (pos <= columns.size())) - { - const auto & column_name = std::next(columns.begin(), pos - 1)->name; - child->children[0] = std::make_shared(column_name); - } - } + auto new_argument = checkPositionalArgument(ast->children.at(0), columns); + if (new_argument) + ast->children[0] = new_argument; } + ASTPtr order_expression = ast->children.at(0); step.addRequiredOutput(order_expression->getColumnName()); if (ast->with_fill) @@ -1319,21 +1317,15 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain } auto & children = select_query->limitBy()->children; + const auto & columns = syntax->source_columns; + for (size_t i = 0; i < children.size(); ++i) { - const auto & columns = syntax->source_columns; - if (auto * ast_literal = typeid_cast(children[i].get())) + if (getContext()->getSettingsRef().enable_positional_arguments) { - auto which = ast_literal->value.getType(); - if (which == Field::Types::UInt64) - { - auto pos = ast_literal->value.get(); - if ((0 < pos) && (pos <= columns.size())) - { - const auto & column_name = std::next(columns.begin(), pos - 1)->name; - children[i] = std::make_shared(column_name); - } - } + auto new_argument = checkPositionalArgument(children[i], columns); + if (new_argument) + children[i] = new_argument; } auto child_name = children[i]->getColumnName(); diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index ed6fdb53eb3..4207cc28e09 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -1,82 +1,75 @@ -- { echo } -select * from test order by 1; -0 1 4 -0 0 2 -0 0 1 -0 1 2 -0 1 2 -0 0 2 -0 1 1 -0 0 3 -0 0 4 -0 1 3 -1 2 4 +set enable_positional_arguments = 1; +drop table if exists test; +create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); +insert into test select number, number, 1 from numbers(2); +insert into test select number, number, 2 from numbers(2); +insert into test select number, number, 3 from numbers(2); +insert into test select number, number, 4 from numbers(2); +insert into test select number, number, 2 from numbers(2); +select * from test where col1 = 1 order by 1; 1 1 1 -1 2 2 1 1 2 -1 2 2 1 1 2 -1 2 1 1 1 3 1 1 4 -1 2 3 -select * from test order by 3; -0 0 1 +select * from test where col2 = 1 order by 3; 1 1 1 -0 1 1 -1 2 1 -0 1 2 -0 0 2 -1 2 2 1 1 2 -0 0 2 1 1 2 -0 1 2 -1 2 2 -0 1 3 -1 2 3 -0 0 3 1 1 3 -0 1 4 -1 2 4 -0 0 4 1 1 4 -select col1, col2 from test group by col1, col2 order by col2; +insert into test select number, number+1, 1 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); +insert into test select number, number+1, 3 from numbers(2); +insert into test select number, number+1, 4 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); +select * from test order by col3 limit 1 by col3; +0 0 1 +0 1 2 +0 0 3 +0 1 4 +select * from test order by 3 limit 1 by 3; +0 0 1 +0 0 2 +0 1 3 +0 0 4 +select col1, col2 from test group by col1, col2 order by col1, col2; 0 0 0 1 1 1 1 2 -select col1, col2 from test group by 1, 2 order by 2; +select col1, col2 from test group by 1, 2 order by 1, 2; 0 0 0 1 1 1 1 2 -select col2, col3 from test group by col3, col2 order by col3; +select col2, col3 from test group by col3, col2 order by col3, col2; 0 1 -2 1 1 1 +2 1 0 2 1 2 2 2 0 3 -2 3 1 3 +2 3 0 4 -2 4 1 4 -select col2, col3 from test group by 3, 2 order by 3; +2 4 +select col2, col3 from test group by 3, 2 order by 3, 2; 0 1 -2 1 1 1 +2 1 0 2 1 2 2 2 0 3 -2 3 1 3 +2 3 0 4 -2 4 1 4 +2 4 select col2 from test group by 2 order by 2; 0 1 @@ -85,13 +78,3 @@ select col2 + 100 from test group by 2 order by col2; 100 101 102 -select * from test order by col3 limit 1 by col3; -0 0 1 -0 1 2 -0 0 3 -0 0 4 -select * from test order by 3 limit 1 by 3; -0 0 1 -0 1 2 -0 0 3 -0 1 4 diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 61867c1b564..1d1c68d56ac 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -1,30 +1,32 @@ -drop table if exists test; -create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); -insert into test select number, number, 1 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number+1, 1 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); -insert into test select number, number, 3 from numbers(2); -insert into test select number, number, 4 from numbers(2); -insert into test select number, number+1, 3 from numbers(2); -insert into test select number, number+1, 4 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); - +-- { echo } set enable_positional_arguments = 1; --- { echo } -select * from test order by 1; -select * from test order by 3; +drop table if exists test; +create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); -select col1, col2 from test group by col1, col2 order by col2; -select col1, col2 from test group by 1, 2 order by 2; +insert into test select number, number, 1 from numbers(2); +insert into test select number, number, 2 from numbers(2); +insert into test select number, number, 3 from numbers(2); +insert into test select number, number, 4 from numbers(2); +insert into test select number, number, 2 from numbers(2); -select col2, col3 from test group by col3, col2 order by col3; -select col2, col3 from test group by 3, 2 order by 3; +select * from test where col1 = 1 order by 1; +select * from test where col2 = 1 order by 3; -select col2 from test group by 2 order by 2; -select col2 + 100 from test group by 2 order by col2; +insert into test select number, number+1, 1 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); +insert into test select number, number+1, 3 from numbers(2); +insert into test select number, number+1, 4 from numbers(2); +insert into test select number, number+1, 2 from numbers(2); select * from test order by col3 limit 1 by col3; select * from test order by 3 limit 1 by 3; + +select col1, col2 from test group by col1, col2 order by col1, col2; +select col1, col2 from test group by 1, 2 order by 1, 2; + +select col2, col3 from test group by col3, col2 order by col3, col2; +select col2, col3 from test group by 3, 2 order by 3, 2; + +select col2 from test group by 2 order by 2; +select col2 + 100 from test group by 2 order by col2; From c50294fda8b48936e6dea57ac4b0bdb4570c3f2b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 10 Aug 2021 17:40:06 +0300 Subject: [PATCH 05/64] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bddd57c45dd..f791763b811 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,7 +114,7 @@ class IColumn; M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ - M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ + M(Bool, enable_positional_arguments, false, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ \ M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "", 0) \ From c1251c89d67c81c692028fca6605981b396b5e70 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 06:17:54 +0000 Subject: [PATCH 06/64] Fix test --- .../02006_test_positional_arguments.reference | 52 +++++++++++-------- .../02006_test_positional_arguments.sql | 18 +++---- 2 files changed, 40 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index 4207cc28e09..e497af0918a 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -2,38 +2,42 @@ set enable_positional_arguments = 1; drop table if exists test; create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); -insert into test select number, number, 1 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number, 3 from numbers(2); +insert into test select number, number, 5 from numbers(2); insert into test select number, number, 4 from numbers(2); +insert into test select number, number, 3 from numbers(2); insert into test select number, number, 2 from numbers(2); -select * from test where col1 = 1 order by 1; -1 1 1 -1 1 2 -1 1 2 -1 1 3 -1 1 4 -select * from test where col2 = 1 order by 3; -1 1 1 +insert into test select number, number, 1 from numbers(2); +select * from test where col1 = 1 order by 3 desc; +1 1 5 +1 1 4 +1 1 3 1 1 2 +1 1 1 +select * from test where col2 = 1 order by 3 asc; +1 1 1 1 1 2 1 1 3 1 1 4 +1 1 5 insert into test select number, number+1, 1 from numbers(2); insert into test select number, number+1, 2 from numbers(2); insert into test select number, number+1, 3 from numbers(2); insert into test select number, number+1, 4 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); -select * from test order by col3 limit 1 by col3; -0 0 1 -0 1 2 -0 0 3 -0 1 4 -select * from test order by 3 limit 1 by 3; +insert into test select number, number+1, 5 from numbers(2); +select * from test order by col1, col2, col3 asc limit 2 by col2; 0 0 1 0 0 2 -0 1 3 -0 0 4 +0 1 1 +0 1 2 +1 2 1 +1 2 2 +select * from test order by 1, 2, 3 asc limit 2 by 2; +0 0 1 +0 0 2 +0 1 1 +0 1 2 +1 2 1 +1 2 2 select col1, col2 from test group by col1, col2 order by col1, col2; 0 0 0 1 @@ -57,6 +61,9 @@ select col2, col3 from test group by col3, col2 order by col3, col2; 0 4 1 4 2 4 +0 5 +1 5 +2 5 select col2, col3 from test group by 3, 2 order by 3, 2; 0 1 1 1 @@ -70,11 +77,14 @@ select col2, col3 from test group by 3, 2 order by 3, 2; 0 4 1 4 2 4 +0 5 +1 5 +2 5 select col2 from test group by 2 order by 2; 0 1 2 -select col2 + 100 from test group by 2 order by col2; +select col2 + 100 from test group by 2 order by 2; 100 101 102 diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 1d1c68d56ac..bbfd1dbfd64 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -4,23 +4,23 @@ set enable_positional_arguments = 1; drop table if exists test; create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); -insert into test select number, number, 1 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number, 3 from numbers(2); +insert into test select number, number, 5 from numbers(2); insert into test select number, number, 4 from numbers(2); +insert into test select number, number, 3 from numbers(2); insert into test select number, number, 2 from numbers(2); +insert into test select number, number, 1 from numbers(2); -select * from test where col1 = 1 order by 1; -select * from test where col2 = 1 order by 3; +select * from test where col1 = 1 order by 3 desc; +select * from test where col2 = 1 order by 3 asc; insert into test select number, number+1, 1 from numbers(2); insert into test select number, number+1, 2 from numbers(2); insert into test select number, number+1, 3 from numbers(2); insert into test select number, number+1, 4 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); +insert into test select number, number+1, 5 from numbers(2); -select * from test order by col3 limit 1 by col3; -select * from test order by 3 limit 1 by 3; +select * from test order by col1, col2, col3 asc limit 2 by col2; +select * from test order by 1, 2, 3 asc limit 2 by 2; select col1, col2 from test group by col1, col2 order by col1, col2; select col1, col2 from test group by 1, 2 order by 1, 2; @@ -29,4 +29,4 @@ select col2, col3 from test group by col3, col2 order by col3, col2; select col2, col3 from test group by 3, 2 order by 3, 2; select col2 from test group by 2 order by 2; -select col2 + 100 from test group by 2 order by col2; +select col2 + 100 from test group by 2 order by 2; From 2da2f07d086fa2394fb3b287b24d3c0c03cd8684 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 11 Aug 2021 10:35:12 +0300 Subject: [PATCH 07/64] Fix style check --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 42699db8b8e..2c77b18aafd 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -165,7 +165,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( static ASTPtr checkPositionalArgument(ASTPtr argument, const NamesAndTypesList & columns) { /// Case when GROUP BY element is position. - /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are contants. + /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are constants. if (auto * ast_literal = typeid_cast(argument.get())) { auto which = ast_literal->value.getType(); From 3012fad56d5708e2bc4c501c423d95b2511a660d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 20:57:40 +0000 Subject: [PATCH 08/64] Fix --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 4 ++-- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 3 --- src/Storages/StoragePostgreSQL.h | 2 -- src/TableFunctions/TableFunctionPostgreSQL.cpp | 1 - 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index c848c784712..259648f4399 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -164,7 +164,7 @@ StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr local_context, const bool table_checked) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, const bool table_checked) const { if (!cache_tables || !cached_tables.count(table_name)) { @@ -179,7 +179,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr auto storage = StoragePostgreSQL::create( StorageID(database_name, table_name), pool, table_name, - ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, local_context, postgres_schema); + ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, postgres_schema); if (cache_tables) cached_tables[table_name] = storage; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 32b9c7e9245..f20e49fe23a 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -98,7 +98,7 @@ StorageExternalDistributed::StorageExternalDistributed( context->getSettingsRef().postgresql_connection_pool_size, context->getSettingsRef().postgresql_connection_pool_wait_timeout); - shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}, context); + shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}); break; } #endif diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index b71f2415fd8..3ea4a03d8e1 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -47,12 +47,10 @@ StoragePostgreSQL::StoragePostgreSQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const String & remote_table_schema_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , remote_table_schema(remote_table_schema_) - , global_context(context_) , pool(std::move(pool_)) { StorageInMemoryMetadata storage_metadata; @@ -347,7 +345,6 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.columns, args.constraints, args.comment, - args.getContext(), remote_table_schema); }, { diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 064fa481f9d..bd5cd317c3d 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -27,7 +27,6 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const std::string & remote_table_schema_ = ""); String getName() const override { return "PostgreSQL"; } @@ -48,7 +47,6 @@ private: String remote_table_name; String remote_table_schema; - ContextPtr global_context; postgres::PoolWithFailoverPtr pool; }; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index ceea29b335b..d701728479b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -37,7 +37,6 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, columns, ConstraintsDescription{}, String{}, - context, remote_table_schema); result->startup(); From b9d8ee125b52bc9f41862e888072d874b5002bd5 Mon Sep 17 00:00:00 2001 From: jasine Date: Sun, 15 Aug 2021 22:26:40 +0800 Subject: [PATCH 09/64] feat: add conversion between snowflake id and datetime --- src/Functions/FunctionSnowflake.h | 208 +++++++++++++++++++ src/Functions/dateTime64ToSnowflake.cpp | 14 ++ src/Functions/dateTimeToSnowflake.cpp | 14 ++ src/Functions/registerFunctions.cpp | 2 + src/Functions/registerFunctionsSnowflake.cpp | 22 ++ src/Functions/snowflakeToDateTime.cpp | 14 ++ src/Functions/snowflakeToDateTime64.cpp | 14 ++ 7 files changed, 288 insertions(+) create mode 100644 src/Functions/FunctionSnowflake.h create mode 100644 src/Functions/dateTime64ToSnowflake.cpp create mode 100644 src/Functions/dateTimeToSnowflake.cpp create mode 100644 src/Functions/registerFunctionsSnowflake.cpp create mode 100644 src/Functions/snowflakeToDateTime.cpp create mode 100644 src/Functions/snowflakeToDateTime64.cpp diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h new file mode 100644 index 00000000000..cf3a91b8e69 --- /dev/null +++ b/src/Functions/FunctionSnowflake.h @@ -0,0 +1,208 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionDateTimeToSnowflake : public IFunction +{ +private: + const char * name; +public: + FunctionDateTimeToSnowflake( const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isVariadic() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateTime(arguments[0].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime", name); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnInt64::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (int64_t(source_data[i])*1000-1288834974657)<<22; + } + + return res_column; + } +}; + + +class FunctionSnowflakeToDateTime : public IFunction +{ +private: + const char * name; +public: + FunctionSnowflakeToDateTime(const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + + if (arguments.size() < 1 || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + + if (!typeid_cast(arguments[0].type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); + + std::string timezone; + if (arguments.size() == 2) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnUInt32::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; + } + + return res_column; + } +}; + + +class FunctionDateTime64ToSnowflake : public IFunction +{ +private: + const char * name; +public: + FunctionDateTime64ToSnowflake( const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isVariadic() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateTime64(arguments[0].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime64", name); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnInt64::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast &>(col).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (source_data[i]-1288834974657)<<22; + } + + return res_column; + } +}; + + +class FunctionSnowflakeToDateTime64 : public IFunction +{ +private: + const char * name; +public: + FunctionSnowflakeToDateTime64(const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + + if (arguments.size() < 1 || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + + if (!typeid_cast(arguments[0].type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); + + std::string timezone; + if (arguments.size() == 2) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + + return std::make_shared(3, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnDecimal::create(input_rows_count, 3); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (source_data[i]>>22)+1288834974657; + } + + return res_column; + } +}; + +} diff --git a/src/Functions/dateTime64ToSnowflake.cpp b/src/Functions/dateTime64ToSnowflake.cpp new file mode 100644 index 00000000000..87e35c25371 --- /dev/null +++ b/src/Functions/dateTime64ToSnowflake.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerDateTime64ToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTime64ToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTime64ToSnowflake")); }); +} + +} diff --git a/src/Functions/dateTimeToSnowflake.cpp b/src/Functions/dateTimeToSnowflake.cpp new file mode 100644 index 00000000000..246f35cc1dc --- /dev/null +++ b/src/Functions/dateTimeToSnowflake.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerDateTimeToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTimeToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTimeToSnowflake")); }); +} + +} diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 7e8f35bc0c4..9b1a7faebbe 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -51,6 +51,7 @@ void registerFunctionBitHammingDistance(FunctionFactory & factory); void registerFunctionTupleHammingDistance(FunctionFactory & factory); void registerFunctionsStringHash(FunctionFactory & factory); void registerFunctionValidateNestedArraySizes(FunctionFactory & factory); +void registerFunctionsSnowflake(FunctionFactory & factory); #if !defined(ARCADIA_BUILD) void registerFunctionBayesAB(FunctionFactory &); #endif @@ -115,6 +116,7 @@ void registerFunctions() registerFunctionTupleHammingDistance(factory); registerFunctionsStringHash(factory); registerFunctionValidateNestedArraySizes(factory); + registerFunctionsSnowflake(factory); #if !defined(ARCADIA_BUILD) registerFunctionBayesAB(factory); diff --git a/src/Functions/registerFunctionsSnowflake.cpp b/src/Functions/registerFunctionsSnowflake.cpp new file mode 100644 index 00000000000..f0c2feddfb5 --- /dev/null +++ b/src/Functions/registerFunctionsSnowflake.cpp @@ -0,0 +1,22 @@ +namespace DB +{ + +class FunctionFactory; + +void registerDateTimeToSnowflake(FunctionFactory &); +void registerSnowflakeToDateTime(FunctionFactory &); + +void registerDateTime64ToSnowflake(FunctionFactory &); +void registerSnowflakeToDateTime64(FunctionFactory &); + + +void registerFunctionsSnowflake(FunctionFactory & factory) +{ + registerDateTimeToSnowflake(factory); + registerSnowflakeToDateTime(factory); + + registerDateTime64ToSnowflake(factory); + registerSnowflakeToDateTime64(factory); +} + +} diff --git a/src/Functions/snowflakeToDateTime.cpp b/src/Functions/snowflakeToDateTime.cpp new file mode 100644 index 00000000000..37f5e07512f --- /dev/null +++ b/src/Functions/snowflakeToDateTime.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerSnowflakeToDateTime(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime")); }); +} + +} diff --git a/src/Functions/snowflakeToDateTime64.cpp b/src/Functions/snowflakeToDateTime64.cpp new file mode 100644 index 00000000000..ef9502a224e --- /dev/null +++ b/src/Functions/snowflakeToDateTime64.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerSnowflakeToDateTime64(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime64", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime64")); }); +} + +} From aa49f76bf0184fb2e4abe9487df7f7c88f8344d4 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 00:49:33 +0800 Subject: [PATCH 10/64] fix: style --- src/Functions/FunctionSnowflake.h | 11 +++++------ src/Functions/registerFunctionsSnowflake.cpp | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index cf3a91b8e69..3dd378e4956 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -25,7 +25,7 @@ class FunctionDateTimeToSnowflake : public IFunction private: const char * name; public: - FunctionDateTimeToSnowflake( const char * name_) + FunctionDateTimeToSnowflake(const char * name_) : name(name_) { } @@ -79,7 +79,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -107,7 +106,7 @@ public: { result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; } - + return res_column; } }; @@ -118,7 +117,7 @@ class FunctionDateTime64ToSnowflake : public IFunction private: const char * name; public: - FunctionDateTime64ToSnowflake( const char * name_) + FunctionDateTime64ToSnowflake(const char * name_) : name(name_) { } @@ -172,7 +171,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - + if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -200,7 +199,7 @@ public: { result_data[i] = (source_data[i]>>22)+1288834974657; } - + return res_column; } }; diff --git a/src/Functions/registerFunctionsSnowflake.cpp b/src/Functions/registerFunctionsSnowflake.cpp index f0c2feddfb5..7a0569ee16a 100644 --- a/src/Functions/registerFunctionsSnowflake.cpp +++ b/src/Functions/registerFunctionsSnowflake.cpp @@ -14,7 +14,7 @@ void registerFunctionsSnowflake(FunctionFactory & factory) { registerDateTimeToSnowflake(factory); registerSnowflakeToDateTime(factory); - + registerDateTime64ToSnowflake(factory); registerSnowflakeToDateTime64(factory); } From c414a3aebf8b819b11dcbf0f541e0d35f9973753 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 17:24:51 +0800 Subject: [PATCH 11/64] feat: add docs and tests --- .../functions/type-conversion-functions.md | 144 ++++++++++++++++++ .../01942_dateTimeToSnowflake.reference | 6 + .../0_stateless/01942_dateTimeToSnowflake.sql | 23 +++ .../01942_snowflakeToDateTime.reference | 3 + .../0_stateless/01942_snowflakeToDateTime.sql | 32 ++++ 5 files changed, 208 insertions(+) create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflake.reference create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflake.sql create mode 100644 tests/queries/0_stateless/01942_snowflakeToDateTime.reference create mode 100644 tests/queries/0_stateless/01942_snowflakeToDateTime.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index efd28def688..5a733f6be23 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1339,3 +1339,147 @@ Result: │ 2,"good" │ └───────────────────────────────────────────┘ ``` + +## snowflakeToDateTime {#snowflakeToDateTime} + +extract time from snowflake id as DateTime format. + +**Syntax** + +``` sql +snowflakeToDateTime(value [, time_zone]) +``` + +**Parameters** + +- `value` — `snowflake id`, Int64 value. +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- value converted to the `DateTime` data type. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC'); +``` + +Result: + +``` text + +┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:57:56 │ +└──────────────────────────────────────────────────────────────────┘ +``` + +## snowflakeToDateTime64 {#snowflakeToDateTime64} + +extract time from snowflake id as DateTime64 format. + +**Syntax** + +``` sql +snowflakeToDateTime64(value [, time_zone]) +``` + +**Parameters** + +- `value` — `snowflake id`, Int64 value. +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- value converted to the `DateTime64` data type. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC'); +``` + +Result: + +``` text + +┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:58:19.841 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +## dateTimeToSnowflake {#dateTimeToSnowflake} + +convert DateTime to the first snowflake id at the giving time. + +**Syntax** + +``` sql +dateTimeToSnowflake(value) +``` + +**Parameters** + +- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md). + + +**Returned value** + +- `value` converted to the `Int64` data type as the first snowflake id at that time. + +**Example** + +Query: + +``` sql +SELECT dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime')); +``` + +Result: + +``` text + +┌─dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime'))─┐ +│ 1426860702823350272 │ +└──────────────────────────────────────────────────────────────┘ +``` + + +## dateTime64ToSnowflake {#dateTime64ToSnowflake} + +convert DateTime64 to the first snowflake id at the giving time. + +**Syntax** + +``` sql +dateTime64ToSnowflake(value) +``` + +**Parameters** + +- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). + + +**Returned value** + +- `value` converted to the `Int64` data type as the first snowflake id at that time. + +**Example** + +Query: + +``` sql +SELECT dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64')); +``` + +Result: + +``` text +┌─dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64'))─┐ +│ 1426860703129534464 │ +└──────────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference new file mode 100644 index 00000000000..dfca3a10eeb --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference @@ -0,0 +1,6 @@ +const column +2021-08-15 18:57:56 1426860702823350272 +2021-08-15 18:57:56.492 1426860704886947840 +non-const column +2021-08-15 18:57:56 1426860702823350272 +2021-08-15 18:57:56.492 1426860704886947840 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql new file mode 100644 index 00000000000..e5895db7004 --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -0,0 +1,23 @@ +-- Error cases +SELECT dateTimeToSnowflake(); -- {serverError 42} +SELECT dateTime64ToSnowflake(); -- {serverError 42} + +SELECT dateTimeToSnowflake('abc'); -- {serverError 43} +SELECT dateTime64ToSnowflake('abc'); -- {serverError 43} + +SELECT dateTimeToSnowflake('abc', 123); -- {serverError 42} +SELECT dateTime64ToSnowflake('abc', 123); -- {serverError 42} + +SELECT 'const column'; +WITH toDateTime('2021-08-15 18:57:56') AS dt +SELECT dt, dateTimeToSnowflake(dt); + +WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64 +SELECT dt64, dateTime64ToSnowflake(dt64); + +SELECT 'non-const column'; +WITH toDateTime('2021-08-15 18:57:56') AS x +SELECT materialize(x) as dt, dateTimeToSnowflake(dt);; + +WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS x +SELECT materialize(x) as dt64, dateTime64ToSnowflake(dt64); diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference new file mode 100644 index 00000000000..bed18023f6a --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference @@ -0,0 +1,3 @@ +const column +UTC 1426860704886947840 2021-08-15 10:57:56 DateTime(\'UTC\') 2021-08-15 10:57:56.492 DateTime64(3, \'UTC\') +Asia/Shanghai 1426860704886947840 2021-08-15 18:57:56 DateTime(\'Asia/Shanghai\') 2021-08-15 18:57:56.492 DateTime64(3, \'Asia/Shanghai\') diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql new file mode 100644 index 00000000000..f6f171afabf --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -0,0 +1,32 @@ +-- -- Error cases +SELECT snowflakeToDateTime(); -- {serverError 42} +SELECT snowflakeToDateTime64(); -- {serverError 42} + +SELECT snowflakeToDateTime('abc'); -- {serverError 43} +SELECT snowflakeToDateTime64('abc'); -- {serverError 43} + +SELECT snowflakeToDateTime('abc', 123); -- {serverError 43} +SELECT snowflakeToDateTime64('abc', 123); -- {serverError 43} + +SELECT 'const column'; +WITH + CAST(1426860704886947840 AS Int64) AS i64, + 'UTC' AS tz +SELECT + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); + +WITH + CAST(1426860704886947840 AS Int64) AS i64, + 'Asia/Shanghai' AS tz +SELECT + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); \ No newline at end of file From 1f21131db680c392e4daeacb47a3ec02b162ef86 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 18:52:10 +0800 Subject: [PATCH 12/64] fix: doc and test --- .../functions/type-conversion-functions.md | 18 ++++++++++-------- .../0_stateless/01942_dateTimeToSnowflake.sql | 8 ++++---- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5a733f6be23..4f1a2d49d23 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1436,16 +1436,17 @@ dateTimeToSnowflake(value) Query: ``` sql -SELECT dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime')); +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt +SELECT dateTimeToSnowflake(dt); ``` Result: ``` text -┌─dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime'))─┐ -│ 1426860702823350272 │ -└──────────────────────────────────────────────────────────────┘ +┌─dateTimeToSnowflake(dt)─┐ +│ 1426860702823350272 │ +└─────────────────────────┘ ``` @@ -1473,13 +1474,14 @@ dateTime64ToSnowflake(value) Query: ``` sql -SELECT dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64')); +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 +SELECT dateTime64ToSnowflake(dt64); ``` Result: ``` text -┌─dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64'))─┐ -│ 1426860703129534464 │ -└──────────────────────────────────────────────────────────────────────┘ +┌─dateTime64ToSnowflake(dt64)─┐ +│ 1426860704886947840 │ +└─────────────────────────────┘ ``` \ No newline at end of file diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index e5895db7004..047d8be7be5 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -9,15 +9,15 @@ SELECT dateTimeToSnowflake('abc', 123); -- {serverError 42} SELECT dateTime64ToSnowflake('abc', 123); -- {serverError 42} SELECT 'const column'; -WITH toDateTime('2021-08-15 18:57:56') AS dt +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dt, dateTimeToSnowflake(dt); -WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64 +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dt64, dateTime64ToSnowflake(dt64); SELECT 'non-const column'; -WITH toDateTime('2021-08-15 18:57:56') AS x +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS x SELECT materialize(x) as dt, dateTimeToSnowflake(dt);; -WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS x +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS x SELECT materialize(x) as dt64, dateTime64ToSnowflake(dt64); From d25d12d7d4ef4c9ff9ed74984d87cc054c836ed7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Aug 2021 12:30:02 +0000 Subject: [PATCH 13/64] better --- src/Compression/LZ4_decompress_faster.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 6972457f11b..72a611e0f43 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,11 +439,14 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while (unlikely(s == 255)); + } while (ip < input_end && unlikely(s == 255)); }; /// Get literal length. + if (unlikely(ip >= input_end)) + return false; + const unsigned token = *ip++; length = token >> 4; if (length == 0x0F) @@ -475,7 +478,7 @@ bool NO_INLINE decompressImpl( ip += length; op = copy_end; - if (unlikely(ip > input_end)) + if (unlikely(ip + 1 >= input_end)) return false; /// Get match offset. From 60dd53784f54af466596df7a1db5190cdef05f8d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Aug 2021 12:46:59 +0000 Subject: [PATCH 14/64] better --- src/Compression/LZ4_decompress_faster.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 72a611e0f43..21a2cc01a12 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -467,13 +467,13 @@ bool NO_INLINE decompressImpl( /// output: xyzHello, w /// ^-op (we will overwrite excessive bytes on next iteration) - { - auto * target = std::min(copy_end, output_end); - wildCopy(op, ip, target); /// Here we can write up to copy_amount - 1 bytes after buffer. + if (unlikely(copy_end > output_end)) + return false; - if (target == output_end) - return true; - } + wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. + + if (copy_end == output_end) + return true; ip += length; op = copy_end; @@ -531,8 +531,9 @@ bool NO_INLINE decompressImpl( copy(op, match); /// copy_amount + copy_amount - 1 - 4 * 2 bytes after buffer. if (length > copy_amount * 2) { - auto * target = std::min(copy_end, output_end); - wildCopy(op + copy_amount, match + copy_amount, target); + if (unlikely(copy_end > output_end)) + return false; + wildCopy(op + copy_amount, match + copy_amount, copy_end); } op = copy_end; From c819880aa786c95b90cdd9664c72c3fb0c94878c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 18 Aug 2021 16:34:57 +0300 Subject: [PATCH 15/64] Add test for clickhouse-keeper start after conversion --- docs/en/operations/clickhouse-keeper.md | 2 +- tests/integration/helpers/cluster.py | 5 + .../configs/keeper_config1.xml | 37 ++++++ .../configs/keeper_config2.xml | 37 ++++++ .../configs/keeper_config3.xml | 37 ++++++ .../configs/logs_conf.xml | 12 ++ .../test.py | 120 ++++++++++++++++++ 7 files changed, 249 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/test.py diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 5fc1baa003c..389cf2361f8 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -114,5 +114,5 @@ Seamlessly migration from ZooKeeper to `clickhouse-keeper` is impossible you hav clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots ``` -4. Copy snapshot to `clickhouse-server` nodes with configured `keeper` or start `clickhouse-keeper` instead of ZooKeeper. Snapshot must persist only on leader node, leader will sync it automatically to other nodes. +4. Copy snapshot to `clickhouse-server` nodes with configured `keeper` or start `clickhouse-keeper` instead of ZooKeeper. Snapshot must persist on all nodes, otherwise empty nodes can be faster and one of them can becamse leader. diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6fe01b5df03..11af6ec6348 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -474,6 +474,11 @@ class ClickHouseCluster: cmd += " client" return cmd + def copy_file_from_container_to_container(self, src_node, src_path, dst_node, dst_path): + fname = os.path.basename(src_path) + run_and_check([f"docker cp {src_node.docker_id}:{src_path} {self.instances_dir}"], shell=True) + run_and_check([f"docker cp {self.instances_dir}/{fname} {dst_node.docker_id}:{dst_path}"], shell=True) + def setup_zookeeper_secure_cmd(self, instance, env_variables, docker_compose_yml_dir): logging.debug('Setup ZooKeeper Secure') zookeeper_docker_compose_path = p.join(docker_compose_yml_dir, 'docker_compose_zookeeper_secure.yml') diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml new file mode 100644 index 00000000000..b3c010fed24 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml new file mode 100644 index 00000000000..f2de2c96f06 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml new file mode 100644 index 00000000000..609c9045259 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/test.py b/tests/integration/test_keeper_snapshot_small_distance/test.py new file mode 100644 index 00000000000..4acd76806b4 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/test.py @@ -0,0 +1,120 @@ +#!/usr/bin/env python3 +##!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from kazoo.client import KazooClient, KazooState +import random +import string +import os +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/keeper_config2.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/keeper_config3.xml'], stay_alive=True) + +def start_zookeeper(node): + node1.exec_in_container(['bash', '-c', '/opt/zookeeper/bin/zkServer.sh start']) + +def stop_zookeeper(node): + node.exec_in_container(['bash', '-c', '/opt/zookeeper/bin/zkServer.sh stop']) + +def clear_zookeeper(node): + node.exec_in_container(['bash', '-c', 'rm -fr /zookeeper/*']) + +def restart_and_clear_zookeeper(node): + stop_zookeeper(node) + clear_zookeeper(node) + start_zookeeper(node) + +def clear_clickhouse_data(node): + node.exec_in_container(['bash', '-c', 'rm -fr /var/lib/clickhouse/coordination/logs/* /var/lib/clickhouse/coordination/snapshots/*']) + +def convert_zookeeper_data(node): + cmd = '/usr/bin/clickhouse keeper-converter --zookeeper-logs-dir /zookeeper/version-2/ --zookeeper-snapshots-dir /zookeeper/version-2/ --output-dir /var/lib/clickhouse/coordination/snapshots' + node.exec_in_container(['bash', '-c', cmd]) + return os.path.join('/var/lib/clickhouse/coordination/snapshots', node.exec_in_container(['bash', '-c', 'ls /var/lib/clickhouse/coordination/snapshots']).strip()) + +def stop_clickhouse(node): + node.stop_clickhouse() + +def start_clickhouse(node): + node.start_clickhouse() + +def copy_zookeeper_data(make_zk_snapshots, node): + stop_zookeeper(node) + + if make_zk_snapshots: # force zookeeper to create snapshot + start_zookeeper(node) + stop_zookeeper(node) + + stop_clickhouse(node) + clear_clickhouse_data(node) + convert_zookeeper_data(node) + start_zookeeper(node) + start_clickhouse(node) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + +def get_genuine_zk(node, timeout=30.0): + _genuine_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":2181", timeout=timeout) + _genuine_zk_instance.start() + return _genuine_zk_instance + + +def test_snapshot_and_load(started_cluster): + restart_and_clear_zookeeper(node1) + genuine_connection = get_genuine_zk(node1) + for node in [node1, node2, node3]: + print("Stop and clear", node.name, "with dockerid", node.docker_id) + stop_clickhouse(node) + clear_clickhouse_data(node) + + for i in range(1000): + genuine_connection.create("/test" + str(i), b"data") + + print("Data loaded to zookeeper") + + stop_zookeeper(node1) + start_zookeeper(node1) + stop_zookeeper(node1) + + print("Data copied to node1") + resulted_path = convert_zookeeper_data(node1) + print("Resulted path", resulted_path) + for node in [node2, node3]: + print("Copy snapshot from", node1.name, "to", node.name) + cluster.copy_file_from_container_to_container(node1, resulted_path, node, '/var/lib/clickhouse/coordination/snapshots') + + print("Starting clickhouses") + + p = Pool(3) + result = p.map_async(start_clickhouse, [node1, node2, node3]) + result.wait() + + print("Loading additional data") + fake_zks = [get_fake_zk(node) for node in [node1, node2, node3]] + for i in range(1000): + fake_zk = random.choice(fake_zks) + try: + fake_zk.create("/test" + str(i + 1000), b"data") + except Exception as ex: + print("Got exception:" + str(ex)) + + print("Final") + fake_zks[0].create("/test10000", b"data") From 4d4dae79f0ecb122b3fb9304ffb85eee7edc0c4d Mon Sep 17 00:00:00 2001 From: ubuntu Date: Thu, 19 Aug 2021 00:47:40 +0800 Subject: [PATCH 16/64] fix: build issue --- .../functions/type-conversion-functions.md | 8 ++-- src/Functions/FunctionSnowflake.h | 44 +++++++++---------- src/Functions/dateTime64ToSnowflake.cpp | 14 ------ src/Functions/dateTimeToSnowflake.cpp | 14 ------ src/Functions/snowflake.cpp | 34 ++++++++++++++ src/Functions/snowflakeToDateTime.cpp | 14 ------ src/Functions/snowflakeToDateTime64.cpp | 14 ------ 7 files changed, 58 insertions(+), 84 deletions(-) delete mode 100644 src/Functions/dateTime64ToSnowflake.cpp delete mode 100644 src/Functions/dateTimeToSnowflake.cpp create mode 100644 src/Functions/snowflake.cpp delete mode 100644 src/Functions/snowflakeToDateTime.cpp delete mode 100644 src/Functions/snowflakeToDateTime64.cpp diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4f1a2d49d23..ad6edaea312 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1342,7 +1342,7 @@ Result: ## snowflakeToDateTime {#snowflakeToDateTime} -extract time from snowflake id as DateTime format. +Extract time from snowflake id as DateTime format. **Syntax** @@ -1378,7 +1378,7 @@ Result: ## snowflakeToDateTime64 {#snowflakeToDateTime64} -extract time from snowflake id as DateTime64 format. +Extract time from snowflake id as DateTime64 format. **Syntax** @@ -1414,7 +1414,7 @@ Result: ## dateTimeToSnowflake {#dateTimeToSnowflake} -convert DateTime to the first snowflake id at the giving time. +Convert DateTime to the first snowflake id at the giving time. **Syntax** @@ -1452,7 +1452,7 @@ Result: ## dateTime64ToSnowflake {#dateTime64ToSnowflake} -convert DateTime64 to the first snowflake id at the giving time. +Convert DateTime64 to the first snowflake id at the giving time. **Syntax** diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 3dd378e4956..3f0f404f7e4 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -12,23 +12,28 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +/** According to Twitter's post on Snowflake, we can extract the timestamp for a snowflake ID by right shifting + * the snowflake ID by 22 bits(10 bits machine ID and 12 bits sequence ID) and adding the Twitter epoch time of 1288834974657. + * https://en.wikipedia.org/wiki/Snowflake_ID + * https://blog.twitter.com/engineering/en_us/a/2010/announcing-snowflake + * https://ws-dl.blogspot.com/2019/08/2019-08-03-tweetedat-finding-tweet.html +*/ +static constexpr long snowflake_epoch = 1288834974657L; +static constexpr int time_shift = 22; class FunctionDateTimeToSnowflake : public IFunction { private: const char * name; + public: - FunctionDateTimeToSnowflake(const char * name_) - : name(name_) - { - } + FunctionDateTimeToSnowflake(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -54,7 +59,7 @@ public: const auto & source_data = typeid_cast(col).getData(); for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (int64_t(source_data[i])*1000-1288834974657)<<22; + result_data[i] = (Int64(source_data[i]) * 1000 - snowflake_epoch) << time_shift; } return res_column; @@ -66,11 +71,9 @@ class FunctionSnowflakeToDateTime : public IFunction { private: const char * name; + public: - FunctionSnowflakeToDateTime(const char * name_) - : name(name_) - { - } + FunctionSnowflakeToDateTime(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -104,9 +107,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; + result_data[i] = ((source_data[i] >> time_shift) + snowflake_epoch) / 1000; } - return res_column; } }; @@ -116,11 +118,9 @@ class FunctionDateTime64ToSnowflake : public IFunction { private: const char * name; + public: - FunctionDateTime64ToSnowflake(const char * name_) - : name(name_) - { - } + FunctionDateTime64ToSnowflake(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -146,7 +146,7 @@ public: const auto & source_data = typeid_cast &>(col).getData(); for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (source_data[i]-1288834974657)<<22; + result_data[i] = (source_data[i] - snowflake_epoch) << time_shift; } return res_column; @@ -158,11 +158,9 @@ class FunctionSnowflakeToDateTime64 : public IFunction { private: const char * name; + public: - FunctionSnowflakeToDateTime64(const char * name_) - : name(name_) - { - } + FunctionSnowflakeToDateTime64(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -171,7 +169,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -197,9 +194,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (source_data[i]>>22)+1288834974657; + result_data[i] = (source_data[i] >> time_shift) + snowflake_epoch; } - return res_column; } }; diff --git a/src/Functions/dateTime64ToSnowflake.cpp b/src/Functions/dateTime64ToSnowflake.cpp deleted file mode 100644 index 87e35c25371..00000000000 --- a/src/Functions/dateTime64ToSnowflake.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDateTime64ToSnowflake(FunctionFactory & factory) -{ - factory.registerFunction("dateTime64ToSnowflake", - [](ContextPtr){ return std::make_unique( - std::make_shared("dateTime64ToSnowflake")); }); -} - -} diff --git a/src/Functions/dateTimeToSnowflake.cpp b/src/Functions/dateTimeToSnowflake.cpp deleted file mode 100644 index 246f35cc1dc..00000000000 --- a/src/Functions/dateTimeToSnowflake.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDateTimeToSnowflake(FunctionFactory & factory) -{ - factory.registerFunction("dateTimeToSnowflake", - [](ContextPtr){ return std::make_unique( - std::make_shared("dateTimeToSnowflake")); }); -} - -} diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp new file mode 100644 index 00000000000..5ac1d229d17 --- /dev/null +++ b/src/Functions/snowflake.cpp @@ -0,0 +1,34 @@ +#include +#include + +namespace DB +{ + +void registerDateTimeToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTimeToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTimeToSnowflake")); }); +} + +void registerDateTime64ToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTime64ToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTime64ToSnowflake")); }); +} + +void registerSnowflakeToDateTime(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime")); }); +} +void registerSnowflakeToDateTime64(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime64", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime64")); }); +} + +} diff --git a/src/Functions/snowflakeToDateTime.cpp b/src/Functions/snowflakeToDateTime.cpp deleted file mode 100644 index 37f5e07512f..00000000000 --- a/src/Functions/snowflakeToDateTime.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerSnowflakeToDateTime(FunctionFactory & factory) -{ - factory.registerFunction("snowflakeToDateTime", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime")); }); -} - -} diff --git a/src/Functions/snowflakeToDateTime64.cpp b/src/Functions/snowflakeToDateTime64.cpp deleted file mode 100644 index ef9502a224e..00000000000 --- a/src/Functions/snowflakeToDateTime64.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerSnowflakeToDateTime64(FunctionFactory & factory) -{ - factory.registerFunction("snowflakeToDateTime64", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime64")); }); -} - -} From 3149e80c5934bbe46bfd11882180141e17873a62 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Aug 2021 09:54:13 +0300 Subject: [PATCH 17/64] Commit missed file --- tests/integration/test_keeper_snapshot_small_distance/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_keeper_snapshot_small_distance/__init__.py diff --git a/tests/integration/test_keeper_snapshot_small_distance/__init__.py b/tests/integration/test_keeper_snapshot_small_distance/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From 3d05014da1d191a389a92a5818c062696fc151e8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Aug 2021 12:54:03 +0300 Subject: [PATCH 18/64] Update FunctionSnowflake.h --- src/Functions/FunctionSnowflake.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 3f0f404f7e4..95d02de3a2b 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -39,6 +39,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -79,6 +80,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -126,6 +128,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -166,6 +169,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { From 0380091be896d1350e2b5d3ebcd871de263ce4c9 Mon Sep 17 00:00:00 2001 From: igomac <714541080@qq.com> Date: Thu, 19 Aug 2021 18:03:43 +0800 Subject: [PATCH 19/64] Update ym-dict-functions.md Fix mistranslated content --- .../functions/ym-dict-functions.md | 54 +++++++++++++------ 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index 429105084dd..fb2c31291ad 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -1,8 +1,8 @@ -# 功能与Yandex的工作。梅特里卡词典 {#functions-for-working-with-yandex-metrica-dictionaries} +# 使用 Yandex.Metrica 字典函数 {#functions-for-working-with-yandex-metrica-dictionaries} -为了使下面的功能正常工作,服务器配置必须指定获取所有Yandex的路径和地址。梅特里卡字典. 字典在任何这些函数的第一次调用时加载。 如果无法加载引用列表,则会引发异常。 +为了使下面的功能正常工作,服务器配置必须指定获取所有 Yandex.Metrica 字典的路径和地址。Yandex.Metrica 字典在任何这些函数的第一次调用时加载。 如果无法加载引用列表,则会引发异常。 -For information about creating reference lists, see the section «Dictionaries». +有关创建引用列表的信息,请参阅 «字典» 部分. ## 多个地理基 {#multiple-geobases} @@ -17,18 +17,18 @@ ClickHouse支持同时使用多个备选地理基(区域层次结构),以 所有字典都在运行时重新加载(每隔一定数量的秒重新加载一次,如builtin_dictionaries_reload_interval config参数中定义,或默认情况下每小时一次)。 但是,可用字典列表在服务器启动时定义一次。 -All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. +所有处理区域的函数都在末尾有一个可选参数—字典键。它被称为地基。 示例: regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txt regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txt regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt -### ツ环板(ョツ嘉ッツ偲青regionシツ氾カツ鉄ツ工ツ渉\]) {#regiontocityid-geobase} +### regionToCity(id[, geobase]) {#regiontocityid-geobase} -Accepts a UInt32 number – the region ID from the Yandex geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. +从 Yandex gebase 接收一个 UInt32 数字类型的 region ID 。如果该区域是一个城市或城市的一部分,它将返回相应城市的区域ID。否则,返回0。 -### 虏茅驴麓卤戮碌禄路戮鲁拢\]) {#regiontoareaid-geobase} +### regionToArea(id[, geobase]) {#regiontoareaid-geobase} 将区域转换为区域(地理数据库中的类型5)。 在所有其他方式,这个功能是一样的 ‘regionToCity’. @@ -84,36 +84,58 @@ LIMIT 15 │ Federation of Bosnia and Herzegovina │ └──────────────────────────────────────────────────────────┘ -### 虏茅驴麓卤戮碌禄路戮鲁拢(陆毛隆隆(803)888-8325\]) {#regiontocountryid-geobase} +### regionToCountry(id[, geobase]) {#regiontocountryid-geobase} 将区域转换为国家。 在所有其他方式,这个功能是一样的 ‘regionToCity’. 示例: `regionToCountry(toUInt32(213)) = 225` 转换莫斯科(213)到俄罗斯(225)。 -### 掳胫((禄脢鹿脷露胫鲁隆鹿((酶-11-16""\[脪陆,ase\]) {#regiontocontinentid-geobase} +### regionToContinent(id[, geobase]) {#regiontocontinentid-geobase} 将区域转换为大陆。 在所有其他方式,这个功能是一样的 ‘regionToCity’. 示例: `regionToContinent(toUInt32(213)) = 10001` 将莫斯科(213)转换为欧亚大陆(10001)。 -### ツ环板(ョツ嘉ッツ偲青regionャツ静ャツ青サツ催ャツ渉\]) {#regiontopopulationid-geobase} +### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} + +查找该区域层次结构中最高的大陆。 + +**语法** + +``` sql +regionToTopContinent(id[, geobase]) +``` + +**参数** + +- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**返回值** + +- Identifier of the top level continent (the latter when you climb the hierarchy of regions). +- 0, if there is none. + +类型: `UInt32`. + +### regionToPopulation(id\[, geobase\]) {#regiontopopulationid-geobase} 获取区域的人口。 -The population can be recorded in files with the geobase. See the section «External dictionaries». +人口可以记录在文件与地球基。请参阅«外部词典»部分。 如果没有为该区域记录人口,则返回0。 在Yandex地理数据库中,可能会为子区域记录人口,但不会为父区域记录人口。 ### regionIn(lhs,rhs\[,地理数据库\]) {#regioninlhs-rhs-geobase} 检查是否 ‘lhs’ 属于一个区域 ‘rhs’ 区域。 如果属于UInt8,则返回等于1的数字,如果不属于则返回0。 -The relationship is reflexive – any region also belongs to itself. +这种关系是反射的——任何地区也属于自己。 -### ツ暗ェツ氾环催ツ団ツ法ツ人\]) {#regionhierarchyid-geobase} +### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase} -Accepts a UInt32 number – the region ID from the Yandex geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. +从 Yandex gebase 接收一个 UInt32 数字类型的 region ID 。返回一个区域id数组,由传递的区域和链上的所有父节点组成。 示例: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. -### 地区名称(id\[,郎\]) {#regiontonameid-lang} +### regionToName(id\[, lang\]) {#regiontonameid-lang} -Accepts a UInt32 number – the region ID from the Yandex geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID doesn’t exist, an empty string is returned. +从 Yandex gebase 接收一个 UInt32 数字类型的 region ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数,则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在,则返回一个空字符串。 `ua` 和 `uk` 都意味着乌克兰。 From 9c22d07deaeccac88a929de8712ea94a59139797 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 19 Aug 2021 14:32:32 +0300 Subject: [PATCH 20/64] Mute current failures --- tests/integration/helpers/cluster.py | 4 +++ .../test.py | 13 +++++++-- .../test.py | 3 ++ tests/integration/test_storage_kafka/test.py | 28 ++++--------------- .../test_storage_kerberized_kafka/test.py | 3 ++ 5 files changed, 27 insertions(+), 24 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6fe01b5df03..c6b44a8b830 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1836,6 +1836,10 @@ class ClickHouseInstance: build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") return "-fsanitize={}".format(sanitizer_name) in build_opts + def is_debug_build(self): + build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + return 'NDEBUG' not in build_opts + def is_built_with_thread_sanitizer(self): return self.is_built_with_sanitizer('thread') diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index d8eb92d96b5..03c85a82c90 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -3,6 +3,7 @@ import os.path import timeit import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV @@ -11,6 +12,8 @@ cluster = ClickHouseCluster(__file__) NODES = {'node' + str(i): None for i in (1, 2)} +IS_DEBUG = False + CREATE_TABLES_SQL = ''' CREATE DATABASE test; @@ -104,6 +107,11 @@ def started_cluster(request): try: cluster.start() + if cluster.instances["node1"].is_debug_build(): + global IS_DEBUG + IS_DEBUG = True + logging.warning("Debug build is too slow to show difference in timings. We disable checks.") + for node_id, node in list(NODES.items()): node.query(CREATE_TABLES_SQL) node.query(INSERT_SQL_TEMPLATE.format(node_id=node_id)) @@ -133,8 +141,9 @@ def _check_timeout_and_exception(node, user, query_base, query): # And it should timeout no faster than: measured_timeout = timeit.default_timer() - start - assert expected_timeout - measured_timeout <= TIMEOUT_MEASUREMENT_EPS - assert measured_timeout - expected_timeout <= TIMEOUT_DIFF_UPPER_BOUND[user][query_base] + if not IS_DEBUG: + assert expected_timeout - measured_timeout <= TIMEOUT_MEASUREMENT_EPS + assert measured_timeout - expected_timeout <= TIMEOUT_DIFF_UPPER_BOUND[user][query_base] # And exception should reflect connection attempts: _check_exception(exception, repeats) diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py index f426c3619a4..f557a69569a 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py @@ -37,6 +37,9 @@ def cluster(): with_hdfs=True) logging.info("Starting cluster...") cluster.start() + if cluster.instances["node1"].is_debug_build(): + # https://github.com/ClickHouse/ClickHouse/issues/27814 + pytest.skip("libhdfs3 calls rand function which does not pass harmful check in debug build") logging.info("Cluster started") fs = HdfsClient(hosts=cluster.hdfs_ip) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index cff2b972983..850112144f8 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -180,28 +180,6 @@ def avro_confluent_message(schema_registry_client, value): }) return serializer.encode_record_with_schema('test_subject', schema, value) -# Fixtures - -@pytest.fixture(scope="module") -def kafka_cluster(): - try: - global kafka_id - cluster.start() - kafka_id = instance.cluster.kafka_docker_id - print(("kafka_id is {}".format(kafka_id))) - yield cluster - - finally: - cluster.shutdown() - -@pytest.fixture(autouse=True) -def kafka_setup_teardown(): - instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() # ensure kafka is alive - kafka_producer_send_heartbeat_msg() # ensure python kafka client is ok - # print("kafka is available - running test") - yield # run test - # Tests def test_kafka_settings_old_syntax(kafka_cluster): @@ -694,6 +672,11 @@ def describe_consumer_group(kafka_cluster, name): def kafka_cluster(): try: cluster.start() + if instance.is_debug_build(): + # https://github.com/ClickHouse/ClickHouse/issues/26547 + pytest.skip("~WriteBufferToKafkaProducer(): Assertion `rows == 0 && chunks.empty()' failed.") + kafka_id = instance.cluster.kafka_docker_id + print(("kafka_id is {}".format(kafka_id))) yield cluster finally: cluster.shutdown() @@ -1124,6 +1107,7 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): + instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 126c52bb1d9..f5fba594022 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -55,6 +55,9 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None): def kafka_cluster(): try: cluster.start() + if instance.is_debug_build(): + # https://github.com/ClickHouse/ClickHouse/issues/27651 + pytest.skip("librdkafka calls system function for kinit which does not pass harmful check in debug build") yield cluster finally: cluster.shutdown() From 1e195cbacf12408784dab71c4a2198d92ab9c247 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 14:48:10 +0300 Subject: [PATCH 21/64] Update LZ4_decompress_faster.cpp --- src/Compression/LZ4_decompress_faster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 21a2cc01a12..ad06900f45c 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,7 +439,7 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while (ip < input_end && unlikely(s == 255)); + } while while (unlikely(s == 255 && ip < input_end)); }; /// Get literal length. From 1697a6fa22d239a570f0570762c864cb6b09f8a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Aug 2021 15:10:55 +0300 Subject: [PATCH 22/64] Fix --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 19560cec9ea..4f31c8dfb52 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -110,7 +110,7 @@ void insertPostgreSQLValue( readDateTime64Text(time, 6, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast &>(column).insertValue(time); + assert_cast(column).insertValue(time); break; } case ExternalResultDescription::ValueType::vtDecimal32: [[fallthrough]]; From 2272c9896c7064cb7a6e3bc3da72c7d55ebe7c82 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 16:53:39 +0300 Subject: [PATCH 23/64] Update LZ4_decompress_faster.cpp --- src/Compression/LZ4_decompress_faster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index ad06900f45c..28a285f00f4 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,7 +439,7 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while while (unlikely(s == 255 && ip < input_end)); + } while (unlikely(s == 255 && ip < input_end)); }; /// Get literal length. From 9fecda940dbb1a093d2693e808bdbe221a33320f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 19 Aug 2021 17:09:44 +0300 Subject: [PATCH 24/64] Fix shutdown of NamedSessionStorage. --- programs/server/Server.cpp | 2 -- src/Interpreters/Context.cpp | 3 +++ src/Interpreters/Session.cpp | 51 ++++++++++++++++++++++-------------- src/Interpreters/Session.h | 5 ++-- 4 files changed, 37 insertions(+), 24 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c30ef52f46a..4d68a8be4e4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -53,7 +53,6 @@ #include #include #include -#include #include #include #include @@ -1429,7 +1428,6 @@ if (ThreadFuzzer::instance().isEffective()) /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. async_metrics.start(); - Session::startupNamedSessions(); { String level_str = config().getString("text_log.level", ""); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a634c19dcd6..348ca84ee6f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -59,6 +59,7 @@ #include #include #include +#include #include #include #include @@ -273,6 +274,8 @@ struct ContextSharedPart return; shutdown_called = true; + Session::shutdownNamedSessions(); + /** After system_logs have been shut down it is guaranteed that no system table gets created or written to. * Note that part changes at shutdown won't be logged to part log. */ diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 7334f2e7640..c0e08395eff 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -54,17 +54,17 @@ class NamedSessionsStorage public: using Key = NamedSessionKey; + static NamedSessionsStorage & instance() + { + static NamedSessionsStorage the_instance; + return the_instance; + } + ~NamedSessionsStorage() { try { - { - std::lock_guard lock{mutex}; - quit = true; - } - - cond.notify_one(); - thread.join(); + shutdown(); } catch (...) { @@ -72,6 +72,20 @@ public: } } + void shutdown() + { + { + std::lock_guard lock{mutex}; + sessions.clear(); + if (!thread.joinable()) + return; + quit = true; + } + + cond.notify_one(); + thread.join(); + } + /// Find existing session or create a new. std::pair, bool> acquireSession( const ContextPtr & global_context, @@ -94,6 +108,10 @@ public: auto context = Context::createCopy(global_context); it = sessions.insert(std::make_pair(key, std::make_shared(key, context, timeout, *this))).first; const auto & session = it->second; + + if (!thread.joinable()) + thread = ThreadFromGlobalPool{&NamedSessionsStorage::cleanThread, this}; + return {session, true}; } else @@ -156,11 +174,9 @@ private: { setThreadName("SessionCleaner"); std::unique_lock lock{mutex}; - - while (true) + while (!quit) { auto interval = closeSessions(lock); - if (cond.wait_for(lock, interval, [this]() -> bool { return quit; })) break; } @@ -208,8 +224,8 @@ private: std::mutex mutex; std::condition_variable cond; - std::atomic quit{false}; - ThreadFromGlobalPool thread{&NamedSessionsStorage::cleanThread, this}; + ThreadFromGlobalPool thread; + bool quit = false; }; @@ -218,13 +234,12 @@ void NamedSessionData::release() parent.releaseSession(*this); } -std::optional Session::named_sessions = std::nullopt; - -void Session::startupNamedSessions() +void Session::shutdownNamedSessions() { - named_sessions.emplace(); + NamedSessionsStorage::instance().shutdown(); } + Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_) : global_context(global_context_) { @@ -317,15 +332,13 @@ ContextMutablePtr Session::makeSessionContext(const String & session_id_, std::c throw Exception("Session context already exists", ErrorCodes::LOGICAL_ERROR); if (query_context_created) throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); - if (!named_sessions) - throw Exception("Support for named sessions is not enabled", ErrorCodes::LOGICAL_ERROR); /// Make a new session context OR /// if the `session_id` and `user_id` were used before then just get a previously created session context. std::shared_ptr new_named_session; bool new_named_session_created = false; std::tie(new_named_session, new_named_session_created) - = named_sessions->acquireSession(global_context, user_id.value_or(UUID{}), session_id_, timeout_, session_check_); + = NamedSessionsStorage::instance().acquireSession(global_context, user_id.value_or(UUID{}), session_id_, timeout_, session_check_); auto new_session_context = new_named_session->context; new_session_context->makeSessionContext(); diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 58370aad2d0..0e816324dad 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -28,9 +28,8 @@ using UserPtr = std::shared_ptr; class Session { public: - /// Allow to use named sessions. The thread will be run to cleanup sessions after timeout has expired. - /// The method must be called at the server startup. - static void startupNamedSessions(); + /// Stops using named sessions. The method must be called at the server shutdown. + static void shutdownNamedSessions(); Session(const ContextPtr & global_context_, ClientInfo::Interface interface_); Session(Session &&); From a9d83c1eec7fd5723a29d0182929d73970a8b713 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 19 Aug 2021 18:15:44 +0300 Subject: [PATCH 25/64] fix postgres like cast with negative numbers --- src/Parsers/ExpressionElementParsers.cpp | 30 ++++++++++++++----- src/Parsers/ExpressionListParsers.cpp | 6 ++-- .../01852_cast_operator_3.reference | 10 +++++++ .../0_stateless/01852_cast_operator_3.sql | 14 +++++++++ .../01852_cast_operator_bad_cases.reference | 8 +++++ .../01852_cast_operator_bad_cases.sh | 10 +++++++ 6 files changed, 69 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/01852_cast_operator_3.reference create mode 100644 tests/queries/0_stateless/01852_cast_operator_3.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 16f2b720b4a..a79b3e51e16 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -850,15 +850,24 @@ static bool isOneOf(TokenType token) return ((token == tokens) || ...); } - bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - /// Parse numbers (including decimals), strings and arrays of them. + /// Parse numbers (including decimals), strings, arrays and tuples of them. const char * data_begin = pos->begin; const char * data_end = pos->end; bool is_string_literal = pos->type == TokenType::StringLiteral; - if (pos->type == TokenType::Number || is_string_literal) + + if (pos->type == TokenType::Minus) + { + ++pos; + if (pos->type != TokenType::Number) + return false; + + data_end = pos->end; + ++pos; + } + else if (pos->type == TokenType::Number || is_string_literal) { ++pos; } @@ -876,7 +885,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::ClosingSquareBracket) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; if (stack.empty() || stack.back() != TokenType::OpeningSquareBracket) return false; @@ -884,7 +893,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::ClosingRoundBracket) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; if (stack.empty() || stack.back() != TokenType::OpeningRoundBracket) return false; @@ -892,10 +901,15 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::Comma) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; } - else if (isOneOf(pos->type)) + else if (pos->type == TokenType::Number) + { + if (!isOneOf(last_token)) + return false; + } + else if (isOneOf(pos->type)) { if (!isOneOf(last_token)) return false; @@ -915,6 +929,8 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!stack.empty()) return false; } + else + return false; ASTPtr type_ast; if (ParserToken(TokenType::DoubleColon).ignore(pos, expected) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 58f5e766905..3aa5c82884b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -664,10 +664,12 @@ bool ParserUnaryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (pos->type == TokenType::Minus) { - ParserLiteral lit_p; Pos begin = pos; + if (ParserCastOperator().parse(pos, node, expected)) + return true; - if (lit_p.parse(pos, node, expected)) + pos = begin; + if (ParserLiteral().parse(pos, node, expected)) return true; pos = begin; diff --git a/tests/queries/0_stateless/01852_cast_operator_3.reference b/tests/queries/0_stateless/01852_cast_operator_3.reference new file mode 100644 index 00000000000..a1e54797d60 --- /dev/null +++ b/tests/queries/0_stateless/01852_cast_operator_3.reference @@ -0,0 +1,10 @@ +-1 +SELECT CAST(\'-1\', \'Int32\') +-0.1 +SELECT CAST(\'-0.1\', \'Decimal(38, 38)\') +-0.111 +SELECT CAST(\'-0.111\', \'Float64\') +[-1,2,-3] +SELECT CAST(\'[-1, 2, -3]\', \'Array(Int32)\') +[-1.1,2,-3] +SELECT CAST(\'[-1.1, 2, -3]\', \'Array(Float64)\') diff --git a/tests/queries/0_stateless/01852_cast_operator_3.sql b/tests/queries/0_stateless/01852_cast_operator_3.sql new file mode 100644 index 00000000000..1ad015a8dc4 --- /dev/null +++ b/tests/queries/0_stateless/01852_cast_operator_3.sql @@ -0,0 +1,14 @@ +SELECT -1::Int32; +EXPLAIN SYNTAX SELECT -1::Int32; + +SELECT -0.1::Decimal(38, 38); +EXPLAIN SYNTAX SELECT -0.1::Decimal(38, 38); + +SELECT -0.111::Float64; +EXPLAIN SYNTAX SELECT -0.111::Float64; + +SELECT [-1, 2, -3]::Array(Int32); +EXPLAIN SYNTAX SELECT [-1, 2, -3]::Array(Int32); + +SELECT [-1.1, 2, -3]::Array(Float64); +EXPLAIN SYNTAX SELECT [-1.1, 2, -3]::Array(Float64); diff --git a/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference b/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference index 2c4517e0eda..b179e5e927a 100644 --- a/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference +++ b/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference @@ -8,3 +8,11 @@ Syntax error Syntax error Syntax error Code: 6 +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error diff --git a/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh b/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh index f2f566b78c4..6c578a0996c 100755 --- a/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh +++ b/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh @@ -15,3 +15,13 @@ $CLICKHOUSE_CLIENT --query="SELECT [1 2]::Array(UInt8)" 2>&1 | grep -o -m1 'Syn $CLICKHOUSE_CLIENT --query="SELECT 1 4::UInt32" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_CLIENT --query="SELECT '1' '4'::UInt32" 2>&1 | grep -o -m1 'Syntax error' $CLICKHOUSE_CLIENT --query="SELECT '1''4'::UInt32" 2>&1 | grep -o -m1 'Code: 6' + +$CLICKHOUSE_CLIENT --query="SELECT ::UInt32" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT ::String" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT -::Int32" 2>&1 | grep -o 'Syntax error' + +$CLICKHOUSE_CLIENT --query="SELECT [1, -]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [1, 3-]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [-, 2]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [--, 2]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [1, 2]-::Array(Int32)" 2>&1 | grep -o 'Syntax error' From 512d66a088a58546a4c36c011b8c8745ddb98c2c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Aug 2021 15:18:39 +0000 Subject: [PATCH 26/64] Fix --- src/Interpreters/ExpressionAnalyzer.cpp | 27 ++-- src/Parsers/ASTSelectQuery.h | 37 +++++ .../02006_test_positional_arguments.reference | 138 +++++++----------- .../02006_test_positional_arguments.sql | 38 ++--- 4 files changed, 120 insertions(+), 120 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2c77b18aafd..8197c0fa0dd 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -162,8 +162,10 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(); } -static ASTPtr checkPositionalArgument(ASTPtr argument, const NamesAndTypesList & columns) +static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression) { + auto columns = select_query->select()->children; + /// Case when GROUP BY element is position. /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are constants. if (auto * ast_literal = typeid_cast(argument.get())) @@ -174,9 +176,19 @@ static ASTPtr checkPositionalArgument(ASTPtr argument, const NamesAndTypesList & auto pos = ast_literal->value.get(); if ((0 < pos) && (pos <= columns.size())) { - const auto & column_name = std::next(columns.begin(), pos - 1)->name; - return std::make_shared(column_name); + --pos; + const auto & column = columns[pos]; + if (const auto * literal_ast = typeid_cast(column.get())) + { + return std::make_shared(literal_ast->name()); + } + else + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal value for positional argument in {}", + ASTSelectQuery::expressionToString(expression)); + } } + /// Do not throw if out of bounds, see appendUnusedGroupByColumn. } } return nullptr; @@ -257,7 +269,6 @@ void ExpressionAnalyzer::analyzeAggregation() { NameSet unique_keys; ASTs & group_asts = select_query->groupBy()->children; - const auto & columns = syntax->source_columns; for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) { @@ -266,7 +277,7 @@ void ExpressionAnalyzer::analyzeAggregation() if (getContext()->getSettingsRef().enable_positional_arguments) { - auto new_argument = checkPositionalArgument(group_asts[i], columns); + auto new_argument = checkPositionalArgument(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY); if (new_argument) group_asts[i] = new_argument; } @@ -1252,7 +1263,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai bool with_fill = false; NameSet order_by_keys; - const auto & columns = syntax->source_columns; for (auto & child : select_query->orderBy()->children) { @@ -1262,7 +1272,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (getContext()->getSettingsRef().enable_positional_arguments) { - auto new_argument = checkPositionalArgument(ast->children.at(0), columns); + auto new_argument = checkPositionalArgument(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY); if (new_argument) ast->children[0] = new_argument; } @@ -1317,13 +1327,12 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain } auto & children = select_query->limitBy()->children; - const auto & columns = syntax->source_columns; for (size_t i = 0; i < children.size(); ++i) { if (getContext()->getSettingsRef().enable_positional_arguments) { - auto new_argument = checkPositionalArgument(children[i], columns); + auto new_argument = checkPositionalArgument(children[i], select_query, ASTSelectQuery::Expression::LIMIT_BY); if (new_argument) children[i] = new_argument; } diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 12817199d13..e439c5edaa5 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -35,6 +35,43 @@ public: SETTINGS }; + static String expressionToString(Expression expr) + { + switch (expr) + { + case Expression::WITH: + return "WITH"; + case Expression::SELECT: + return "SELECT"; + case Expression::TABLES: + return "TABLES"; + case Expression::PREWHERE: + return "PREWHERE"; + case Expression::WHERE: + return "WHERE"; + case Expression::GROUP_BY: + return "GROUP BY"; + case Expression::HAVING: + return "HAVING"; + case Expression::WINDOW: + return "WINDOW"; + case Expression::ORDER_BY: + return "ORDER BY"; + case Expression::LIMIT_BY_OFFSET: + return "LIMIT BY OFFSET"; + case Expression::LIMIT_BY_LENGTH: + return "LIMIT BY LENGTH"; + case Expression::LIMIT_BY: + return "LIMIT BY"; + case Expression::LIMIT_OFFSET: + return "LIMIT OFFSET"; + case Expression::LIMIT_LENGTH: + return "LIMIT LENGTH"; + case Expression::SETTINGS: + return "SETTINGS"; + } + } + /** Get the text that identifies this element. */ String getID(char) const override { return "SelectQuery"; } diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index e497af0918a..a8e8ccec100 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -1,90 +1,50 @@ -- { echo } -set enable_positional_arguments = 1; -drop table if exists test; -create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); -insert into test select number, number, 5 from numbers(2); -insert into test select number, number, 4 from numbers(2); -insert into test select number, number, 3 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number, 1 from numbers(2); -select * from test where col1 = 1 order by 3 desc; -1 1 5 -1 1 4 -1 1 3 -1 1 2 -1 1 1 -select * from test where col2 = 1 order by 3 asc; -1 1 1 -1 1 2 -1 1 3 -1 1 4 -1 1 5 -insert into test select number, number+1, 1 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); -insert into test select number, number+1, 3 from numbers(2); -insert into test select number, number+1, 4 from numbers(2); -insert into test select number, number+1, 5 from numbers(2); -select * from test order by col1, col2, col3 asc limit 2 by col2; -0 0 1 -0 0 2 -0 1 1 -0 1 2 -1 2 1 -1 2 2 -select * from test order by 1, 2, 3 asc limit 2 by 2; -0 0 1 -0 0 2 -0 1 1 -0 1 2 -1 2 1 -1 2 2 -select col1, col2 from test group by col1, col2 order by col1, col2; -0 0 -0 1 -1 1 -1 2 -select col1, col2 from test group by 1, 2 order by 1, 2; -0 0 -0 1 -1 1 -1 2 -select col2, col3 from test group by col3, col2 order by col3, col2; -0 1 -1 1 -2 1 -0 2 -1 2 -2 2 -0 3 -1 3 -2 3 -0 4 -1 4 -2 4 -0 5 -1 5 -2 5 -select col2, col3 from test group by 3, 2 order by 3, 2; -0 1 -1 1 -2 1 -0 2 -1 2 -2 2 -0 3 -1 3 -2 3 -0 4 -1 4 -2 4 -0 5 -1 5 -2 5 -select col2 from test group by 2 order by 2; -0 -1 -2 -select col2 + 100 from test group by 2 order by 2; -100 -101 -102 +select x3, x2, x1 from test order by 1; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by x3; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by 1 desc; +100 10 1 +10 1 10 +1 100 100 +select x3, x2, x1 from test order by x3 desc; +100 10 1 +10 1 10 +1 100 100 +insert into test values (1, 10, 200), (10, 1, 200), (100, 100, 1); +select x3, x2 from test group by x3, x2; +200 1 +10 1 +200 10 +1 100 +100 10 +select x3, x2 from test group by 1, 2; +200 1 +10 1 +200 10 +1 100 +100 10 +select x1, x2, x3 from test order by x3 limit 1 by x3; +100 100 1 +10 1 10 +1 10 100 +1 10 200 +select x1, x2, x3 from test order by 3 limit 1 by 3; +100 100 1 +10 1 10 +1 10 100 +1 10 200 +select x1, x2, x3 from test order by x3 limit 1 by x1; +100 100 1 +10 1 10 +1 10 100 +select x1, x2, x3 from test order by 3 limit 1 by 1; +100 100 1 +10 1 10 +1 10 100 +select max(x3), max(x2), max(x1) from test group by 1; -- { serverError 43 } +select max(x1) from test order by 1; -- { serverError 43 } diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index bbfd1dbfd64..dc45b288016 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -1,32 +1,26 @@ --- { echo } set enable_positional_arguments = 1; drop table if exists test; -create table test (col1 Int32, col2 Int32, col3 Int32) engine = Memory(); +create table test(x1 Int, x2 Int, x3 Int) engine=Memory(); +insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); -insert into test select number, number, 5 from numbers(2); -insert into test select number, number, 4 from numbers(2); -insert into test select number, number, 3 from numbers(2); -insert into test select number, number, 2 from numbers(2); -insert into test select number, number, 1 from numbers(2); +-- { echo } +select x3, x2, x1 from test order by 1; +select x3, x2, x1 from test order by x3; -select * from test where col1 = 1 order by 3 desc; -select * from test where col2 = 1 order by 3 asc; +select x3, x2, x1 from test order by 1 desc; +select x3, x2, x1 from test order by x3 desc; -insert into test select number, number+1, 1 from numbers(2); -insert into test select number, number+1, 2 from numbers(2); -insert into test select number, number+1, 3 from numbers(2); -insert into test select number, number+1, 4 from numbers(2); -insert into test select number, number+1, 5 from numbers(2); +insert into test values (1, 10, 200), (10, 1, 200), (100, 100, 1); +select x3, x2 from test group by x3, x2; +select x3, x2 from test group by 1, 2; -select * from test order by col1, col2, col3 asc limit 2 by col2; -select * from test order by 1, 2, 3 asc limit 2 by 2; +select x1, x2, x3 from test order by x3 limit 1 by x3; +select x1, x2, x3 from test order by 3 limit 1 by 3; +select x1, x2, x3 from test order by x3 limit 1 by x1; +select x1, x2, x3 from test order by 3 limit 1 by 1; -select col1, col2 from test group by col1, col2 order by col1, col2; -select col1, col2 from test group by 1, 2 order by 1, 2; +select max(x3), max(x2), max(x1) from test group by 1; -- { serverError 43 } +select max(x1) from test order by 1; -- { serverError 43 } -select col2, col3 from test group by col3, col2 order by col3, col2; -select col2, col3 from test group by 3, 2 order by 3, 2; -select col2 from test group by 2 order by 2; -select col2 + 100 from test group by 2 order by 2; From ec9323f4bd4aa86982bfb3639987f9f6c7501e72 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Aug 2021 15:47:26 +0000 Subject: [PATCH 27/64] Fix review comment --- src/Interpreters/ExpressionAnalyzer.cpp | 5 ++--- src/Interpreters/TreeOptimizer.cpp | 19 +++++++++++++++++-- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8197c0fa0dd..94dadddcf13 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -174,10 +174,9 @@ static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * se if (which == Field::Types::UInt64) { auto pos = ast_literal->value.get(); - if ((0 < pos) && (pos <= columns.size())) + if (pos > 0 && pos <= columns.size()) { - --pos; - const auto & column = columns[pos]; + const auto & column = columns[--pos]; if (const auto * literal_ast = typeid_cast(column.get())) { return std::make_shared(literal_ast->name()); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 8257e54defc..1d2b0670f8c 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -168,9 +168,24 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum std::back_inserter(group_exprs), is_literal ); } - else if (is_literal(group_exprs[i]) && !settings.enable_positional_arguments) + else if (is_literal(group_exprs[i])) { - remove_expr_at_index(i); + bool keep_position = false; + if (settings.enable_positional_arguments) + { + const auto & value = group_exprs[i]->as ()->value; + if (value.getType() == Field::Types::UInt64) + { + auto pos = value.get(); + if (pos > 0 && pos <= select_query->children.size()) + keep_position = true; + } + } + + if (keep_position) + ++i; + else + remove_expr_at_index(i); } else { From 7e2be8f93c234ec0a8a3af796b79b644cf78f635 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 19:02:26 +0300 Subject: [PATCH 28/64] blog article about perf tests --- website/blog/en/2021/performance-test-1.md | 83 ++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 website/blog/en/2021/performance-test-1.md diff --git a/website/blog/en/2021/performance-test-1.md b/website/blog/en/2021/performance-test-1.md new file mode 100644 index 00000000000..8c2259e59d7 --- /dev/null +++ b/website/blog/en/2021/performance-test-1.md @@ -0,0 +1,83 @@ +--- +title: 'Testing the Performance of ClickHouse' +image: 'https://blog-images.clickhouse.tech/en/2021/code-review/two-ducks.jpg' +date: '2021-08-19' +author: '[Alexander Kuzmenkov](https://github.com/akuzm)' +tags: ['testing', 'performance'] +--- + +One of the main selling points of ClickHouse is that it's very fast, in many cases utilizing the hardware up to the theoretical limits. This was noted by many independent benchmark such as [this one](http://brandonharris.io/redshift-clickhouse-time-series/). This speed boils down to a right combination of architectural choices and algorithmic optimizations, sprinkled with a dash of pixie dust. There is an [overview of these factors](https://clickhouse.tech/docs/en/faq/general/why-clickhouse-is-so-fast) on our website, or a talk by the ClickHouse lead developer Alexey Milovidov ["The secrets of ClickHouse performance optimizations"](https://www.youtube.com/watch?v=ZOZQCQEtrz8). But this is a static picture of "how the things are". Software is a living and changing organism, and ClickHouse is changing very fast — to give you a scale, in July 2021 we merged 319 pull requests made by 60 different authors ([live statistics here](https://gh-api.clickhouse.tech/play?user=play#c2VsZWN0IGRhdGVfdHJ1bmMoJ21vbnRoJywgY3JlYXRlZF9hdCkgbW9udGgsIHVuaXEoY3JlYXRvcl91c2VyX2xvZ2luKSBhdXRob3JzLCB1bmlxKG51bWJlcikgcHJzIGZyb20gZ2l0aHViX2V2ZW50cyB3aGVyZSByZXBvX25hbWUgPSAnQ2xpY2tIb3VzZS9DbGlja0hvdXNlJyBhbmQgbm90IGhhc0FueShsYWJlbHMsIFsncHItYmFja3BvcnQnLCAncHItZG9jdW1lbnRhdGlvbicsICdwci1jaGVycnlwaWNrJ10pIGFuZCBtZXJnZWQgYW5kIGNyZWF0ZWRfYXQgYmV0d2VlbiAnMjAyMC0wOS0wMScgYW5kICcyMDIxLTA5LTAxJyBncm91cCBieSBtb250aA==)). Any quality that is not actively selected for is going to be lost in this endless stream of changes, and the performance is no exception. For this reason, we have to have some process that allows us to ensure than ClickHouse always stays fast. + +# Measuring and Comparing the Performance + +How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test varions layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). + +Micro-benchmarks are normally ran by a developer while working on the code, but it is not practical to manually run the entire battery of the end-to-end tests for each change. We use an automated system that does this for each pull request as part of continuous integration checks. It measures whether the code changes introduced by a pull request influenced the performance, for which kinds of queries and by how much, and alerts the developer if there is a regression. Here is how a typical report looks. + + + +To talk about "changes in performance", we first have to measure this performance. The most natural measure for a single query is elapsed time. It is susceptible to random variations, so we have to take several measurements and average them in some way. From the application point of view, the most interesting statistic is maximum. We want to guarantee that e.g. an analytical dashboard built on ClickHouse is responsive. However, the query time can grow almost without limit due to random factor such as sudden disk load spikes or network delays, so using the maximum is not practical. The minimum is also interesting — after all, there is a theoretical bound on it. We know that the particular algorithm can run only so fast on the particular hardware, in ideal conditions. But if we only look at the minimum, we are going to miss cases where some runs of the query are slow and some are not (e.g. boundary effects in some cache). So we compromise by measuring the median. It is a robust statistic that is reasonably sensitive to outliers and stable enough against noise. + +After measuring the performance, how do we determine that it has changed? Due to various random and systematic factors, the query time always drifts, so the number always changes, but the question is whether this change is meaningful. If we have an old version of the server, and a new version of the server, are they going to consistently give a different result for this query, or was it just a fluke? To answer this, we have to employ some statistical method. The core idea of these methods is comparing the observed values to some reference distribution, and deciding whether what we observed can plausibly belong to this distribution, or, on the contrary, it cannot, which means that the performance characteristics of the patched server are indeed different. + +Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works for well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. Howewer, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this troughput. + +For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance task goes to a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. + +We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is thorougly described in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. + +This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of mediand _D_: +1. abs(_D_) <= _T_ — the changes are not statistically significant, +2. abs(_D_) <= 5% — the changes are too small to be important, +3. abs(_T_) >= 10% — the test query has excessive run time variance that leads to poor sensitivity, +4. finally, abs(_D_) >= _T_ and abs(_D_) >= 5% — there are statistically significant changes of significant magnitude. + +The most interesting case are the unstable queries _(3)_. When the query run time varies significantly even on the same version of server, it means we won't be able to detect any changes of performance of this query, because these changes are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? + +# Understanding the Reasons Behind the Changes + +An investigation of code performance often starts with applying a profiler. On Linux, you would use `perf`, a sampling profiler that periodically collects the stack trace of the process, so that you can then see an aggregate picture of where your program spends the most time. In ClickHouse, we actually have a built-in sampling profiler that saves results into a system table, so no external tools are needed. It can be enabled for all queries or for a particular one, by passing the settings [as described in the docs](https://clickhouse.tech/docs/en/operations/optimizing-performance/sampling-query-profiler/). It is on by default, so if you use a recent version of ClickHouse, you already have a combined profile of your production server load. To visualize it, we can use a well-known script for building [flamegraphs](https://github.com/brendangregg/FlameGraph): +``` +clickhouse-client -q "SELECT + arrayStringConcat( + arrayMap( + x -> concat(splitByChar('/', addressToLine(x))[-1], + '#', demangle(addressToSymbol(x))), + trace), + ';') AS stack, + count(*) AS samples +FROM system.trace_log +WHERE trace_type = 'Real' +AND query_id = '4aac5305-b27f-4a5a-91c3-61c0cf52ec2a' +GROUP BY trace" \ +| flamegraph.pl + +``` + +As an example, let's use the test run we've seen above. The tested [pull request](https://github.com/ClickHouse/ClickHouse/pull/26248) is supposed to speed up the `sum` aggregate function for nullable integer types. Let's look at the query #8 of the test 'sum': `SELECT sum(toNullable(number)) FROM numbers(100000000)`. The test system reported that its performance increased by 38.5%, and built a "differential" variant of flamegraph for it, that shows the relative time spent in different functions. We can see that the function that calculates the sum, `DB::AggregateFunctionSumData::addManyNotNull`, now takes 15% less time. + + + +To get more leads into why the performance has changed, we can check how the various query metrics have changed between the old and the new servers. This includes all the metrics from `system.query_log.ProfileEvents`, such as `SelectedRows` or `RealTimeMicroseconds`. ClickHouse also tracks the hardware CPU metrics such as the number of branch or cache misses, using the Linux `perf_event_open` API. After downloading the test output archive, we can use a simple ad hoc [script](https://gist.github.com/akuzm/bb28a442f882349e0a5ec2b5262b97d0) to build some statistics and graphs of these metrics. + + + +This graph shows the number of branch instructions per second, on the old and the new server. We can see that the number of branch instructions has dramatically decreased, which might explain the performance difference. The tested pull request removes some `if`s and replaces them with multiplication, so this explanation sounds plausible. + +While side-to-side comparison is more robust against the systemic errors, the historical data is still very valuable for finding where a regression was introduced or investigating the unstable test queries. This is why we save the results of all test runs into a ClickHouse database. Let's consider the same query #8 from the `sum` test. We can build the history of performance changes with this [SQL query](https://play-ci.clickhouse.tech/play?user=play#V0lUSCAwLjA1IEFTIHMKU0VMRUNUIG9sZF9zaGEsIG5ld19zaGEsIGV2ZW50X3RpbWUsIG1lc3NhZ2UsIG9sZF92YWx1ZSBBUyBgb2xkIHNlcnZlcmAsICAgbmV3X3ZhbHVlIEFTIGBuZXcgc2VydmVyYCwgYmVmb3JlIEFTIGBwcmV2IDExIHJ1bnNgLCBhZnRlciBBUyBgbmV4dCAxMSBydW5zYCwgICAgZGlmZiBBUyBgZGlmZiwgcmF0aW9gLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsIEFTIGBzdGF0IHRocmVzaG9sZCwgcmF0aW8sIGhpc3RvcmljYWxgLCBzdGF0X3RocmVzaG9sZCBBUyBgc3RhdCB0aHJlc2hvbGQsIHJhdGlvLCBwZXItcnVuYCwgY3B1X21vZGVsLHF1ZXJ5X2Rpc3BsYXlfbmFtZQpGUk9NIAooU0VMRUNUICosIHJ1bl9hdHRyaWJ1dGVzX3YxLnZhbHVlIEFTIGNwdV9tb2RlbCwKICAgICAgICBtZWRpYW4ob2xkX3ZhbHVlKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIDExIFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIGJlZm9yZSwKICAgICAgICBtZWRpYW4obmV3X3ZhbHVlKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIENVUlJFTlQgUk9XIEFORCAxMSBGT0xMT1dJTkcpIEFTIGFmdGVyLAogICAgICAgIHF1YW50aWxlRXhhY3QoMC45NSkoYWJzKGRpZmYpKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIDM3IFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIHN0YXRfdGhyZXNob2xkX2hpc3RvcmljYWwKICAgIEZST00gcGVyZnRlc3QucXVlcnlfbWV0cmljc192MgogICAgTEVGVCBKT0lOIHBlcmZ0ZXN0LnJ1bl9hdHRyaWJ1dGVzX3YxIFVTSU5HIChvbGRfc2hhLCBuZXdfc2hhKQogICAgV0hFUkUgKGF0dHJpYnV0ZSA9ICdsc2NwdS1tb2RlbC1uYW1lJykgQU5EIChtZXRyaWMgPSAnY2xpZW50X3RpbWUnKQogICAgICAgIC0tIG9ubHkgZm9yIGNvbW1pdHMgaW4gbWFzdGVyCiAgICAgICAgQU5EIChwcl9udW1iZXIgPSAwKQogICAgICAgIC0tIHNlbGVjdCB0aGUgcXVlcmllcyB3ZSBhcmUgaW50ZXJlc3RlZCBpbgogICAgICAgIEFORCAodGVzdCA9ICdzdW0nKSBBTkQgKHF1ZXJ5X2luZGV4ID0gOCkKKSBBUyB0CkFOWSBMRUZUIEpPSU4gYGdoLWRhdGFgLmNvbW1pdHMgT04gbmV3X3NoYSA9IHNoYQpXSEVSRQogICAgLS0gQ2hlY2sgZm9yIGEgcGVyc2lzdGVudCBhbmQgc2lnbmlmaWNhbnQgY2hhbmdlIGluIHF1ZXJ5IHJ1biB0aW1lLCBpbnRyb2R1Y2VkIGJ5IGEgY29tbWl0OgogICAgLS0gMSkgb24gYSBoaXN0b3JpY2FsIGdyYXBoIG9mIHF1ZXJ5IHJ1biB0aW1lLCB0aGVyZSBpcyBhIHN0ZXAgYmV0d2VlbiB0aGUgYWRqYWNlbnQgY29tbWl0cywKICAgIC0tIHRoYXQgaXMgaGlnaGVyIHRoYW4gdGhlIG5vcm1hbCB2YXJpYW5jZSwKICAgICgoKGFicyhhZnRlciAtIGJlZm9yZSkgLyBpZihhZnRlciA+IGJlZm9yZSwgYWZ0ZXIsIGJlZm9yZSkpIEFTIHN0ZXBfaGVpZ2h0KSA+PSBncmVhdGVzdChzLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsKSkKICAgIC0tIDIpIGluIHNpZGUtdG8tc2lkZSBjb21wYXJpc29uIG9mIHRoZXNlIHR3byBjb21taXRzLCB0aGVyZSB3YXMgYSBzdGF0aXN0aWNhbGx5IHNpZ25pZmljYW50IGRpZmZlcmVuY2UKICAgIC0tIHRoYXQgaXMgYWxzbyBoaWdoZXIgdGhhbiB0aGUgbm9ybWFsIHZhcmlhbmNlLAogICAgICAgIEFORCAoYWJzKGRpZmYpID49IGdyZWF0ZXN0KHN0YXRfdGhyZXNob2xkLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsLCBzKSkKICAgIC0tIDMpIGZpbmFsbHksIHRoaXMgc2lkZS10by1zaWRlIGRpZmZlcmVuY2UgaXMgb2YgbWFnbml0dWRlIGNvbXBhcmFibGUgdG8gdGhlIHN0ZXAgaW4gaGlzdG9yaWNhbCBncmFwaHMuCiAgICAgICAgQU5EIChhYnMoZGlmZikgPj0gKDAuNyAqIHN0ZXBfaGVpZ2h0KSkKb3JkZXIgYnkgZXZlbnRfdGltZSBkZXNjCmZvcm1hdCBWZXJ0aWNhbAoKCg==) to the live ClickHouse CI database. Open the link and run the query so that you can examine the query and see the result for yourself. There were three significant changes of performance throughout the test history. The most recent is a speedup in PR we started with. The second speedup is related to fully switching to clang 11. Curiously, there is also a small slowdown introduced by a PR that was supposed to speed it up instead. + +# Usability Considerations + +Regardless of how it works inside, a test system must be actually usable as a part of the development process. First and foremost, the false positive rate should be as low as possible. False positives are costly to investigate, and if they happen often, developers perceive the test as generally unreliable and tend to ignore the true positives as well. The test must also provide a concise report that makes it obvious what went wrong. We have not really succeeded in this. This test has many more failure modes than a plain functional test, and worse, some of these failures are quantitative, not binary. Much of the complexity is essential, and we try to alleviate it by providing good documentation and linking to the relevant parts of it right from the report page. Another important thing is that the user must be able to investigate a problematic query post-mortem, without running it again locally. This is why we try to export every metric and every intermediate result we have, in easily-manipulated plain text formats. + +Organizationally, it is hard to prevent devolving into a system that does a lot of busywork to just show a green check without giving any insight. I like to call this process "mining the green check", by analogy to cryptocurrencies. Our previous system did just that. It used increasingly complex heuristics tailored to each test query to prevent false positives, restarted itself many times if the results didn't look good, and so on. Ultimately, it wasted a lot of processing power without giving the real picture of the server performance. If you wanted to be sure that the performance did or did not change, you had to recheck by hand. This sorry state is the result of how the incentives are aligned around development — most of the time, the developers just want to merge their pull requests and not be bothered by some obscure test failures. Writing a good performance test query is also not always simple. Just any other query won't do — it has to give predictable performance, be not too fast and not too slow, actually measure something, and so on. After gathering more precise statistics, we discovered that several hundred of our test queries don't measure anything meaningful, e.g. they give a result that varies by 100% between runs. Another problem is that the performance often changes in statistically significant ways (true positive) with no relevant code changes (due to e.g. random differences in layout of the executable). Given all these difficulties, a working performance test system is bound to add noticeable friction to the development process. Most of the "obvious" ways to remove this friction ultimately boil down to "mining the green check". + +Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unresonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. + +This concludes the overview of the ClickHouse performance test system. Stay tuned for the next article where we will discuss the most problematic kind of a performance test failure — the unstable query run time. + +_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm)_ + +References: + +1. Box, Hunter, Hunter, 2005. Statistics for experimenters, p. 78: A Randomized Design Used in the Comparison of Standard and Modified Fertilizer Mixtures for Tomato Plants. \ No newline at end of file From 6fa539e9529465c6a5a9beb5205097908928da92 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 20:17:25 +0300 Subject: [PATCH 29/64] fixes --- website/blog/en/2021/performance-test-1.md | 26 +++++++++++----------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/website/blog/en/2021/performance-test-1.md b/website/blog/en/2021/performance-test-1.md index 8c2259e59d7..3d15a7ea3ec 100644 --- a/website/blog/en/2021/performance-test-1.md +++ b/website/blog/en/2021/performance-test-1.md @@ -1,6 +1,6 @@ --- title: 'Testing the Performance of ClickHouse' -image: 'https://blog-images.clickhouse.tech/en/2021/code-review/two-ducks.jpg' +image: 'https://blog-images.clickhouse.tech/en/2021/performance-testing-1/chebu-crop.jpg' date: '2021-08-19' author: '[Alexander Kuzmenkov](https://github.com/akuzm)' tags: ['testing', 'performance'] @@ -10,7 +10,7 @@ One of the main selling points of ClickHouse is that it's very fast, in many cas # Measuring and Comparing the Performance -How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test varions layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). +How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test various layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). Micro-benchmarks are normally ran by a developer while working on the code, but it is not practical to manually run the entire battery of the end-to-end tests for each change. We use an automated system that does this for each pull request as part of continuous integration checks. It measures whether the code changes introduced by a pull request influenced the performance, for which kinds of queries and by how much, and alerts the developer if there is a regression. Here is how a typical report looks. @@ -20,19 +20,19 @@ To talk about "changes in performance", we first have to measure this performanc After measuring the performance, how do we determine that it has changed? Due to various random and systematic factors, the query time always drifts, so the number always changes, but the question is whether this change is meaningful. If we have an old version of the server, and a new version of the server, are they going to consistently give a different result for this query, or was it just a fluke? To answer this, we have to employ some statistical method. The core idea of these methods is comparing the observed values to some reference distribution, and deciding whether what we observed can plausibly belong to this distribution, or, on the contrary, it cannot, which means that the performance characteristics of the patched server are indeed different. -Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works for well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. Howewer, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this troughput. +Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. However, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this throughput. -For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance task goes to a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. +For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance testing task ends up on a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. -We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is thorougly described in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. +We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is described in detail in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. -This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of mediand _D_: -1. abs(_D_) <= _T_ — the changes are not statistically significant, -2. abs(_D_) <= 5% — the changes are too small to be important, -3. abs(_T_) >= 10% — the test query has excessive run time variance that leads to poor sensitivity, -4. finally, abs(_D_) >= _T_ and abs(_D_) >= 5% — there are statistically significant changes of significant magnitude. +This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server, that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of medians _D_: +1. _abs(D) <= T_ — the changes are not statistically significant, +2. _abs(D) <= 5%_ — the changes are too small to be important, +3. _abs(T) >= 10%_ — the test query has excessive run time variance that leads to poor sensitivity, +4. finally, _abs(D) >= T and abs(D) >= 5%_ — there are statistically significant changes of significant magnitude. -The most interesting case are the unstable queries _(3)_. When the query run time varies significantly even on the same version of server, it means we won't be able to detect any changes of performance of this query, because these changes are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? +The most interesting case are the unstable queries _(3)_. When the elapsed time changes significantly between runs even on the same version of server, it means we won't be able to detect the changes of performance, because they are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? # Understanding the Reasons Behind the Changes @@ -72,11 +72,11 @@ Regardless of how it works inside, a test system must be actually usable as a pa Organizationally, it is hard to prevent devolving into a system that does a lot of busywork to just show a green check without giving any insight. I like to call this process "mining the green check", by analogy to cryptocurrencies. Our previous system did just that. It used increasingly complex heuristics tailored to each test query to prevent false positives, restarted itself many times if the results didn't look good, and so on. Ultimately, it wasted a lot of processing power without giving the real picture of the server performance. If you wanted to be sure that the performance did or did not change, you had to recheck by hand. This sorry state is the result of how the incentives are aligned around development — most of the time, the developers just want to merge their pull requests and not be bothered by some obscure test failures. Writing a good performance test query is also not always simple. Just any other query won't do — it has to give predictable performance, be not too fast and not too slow, actually measure something, and so on. After gathering more precise statistics, we discovered that several hundred of our test queries don't measure anything meaningful, e.g. they give a result that varies by 100% between runs. Another problem is that the performance often changes in statistically significant ways (true positive) with no relevant code changes (due to e.g. random differences in layout of the executable). Given all these difficulties, a working performance test system is bound to add noticeable friction to the development process. Most of the "obvious" ways to remove this friction ultimately boil down to "mining the green check". -Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unresonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. +Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unreasonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. This concludes the overview of the ClickHouse performance test system. Stay tuned for the next article where we will discuss the most problematic kind of a performance test failure — the unstable query run time. -_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm)_ +_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm). Title photo by [Alexander Tokmakov](https://github.com/tavplubix)_ References: From 9013892a965d186e229bb1e64c28cd125d5cdd8f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 20:56:16 +0300 Subject: [PATCH 30/64] make the sql-standard window functions case insensitive --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1b8406682ea..132bdb7b327 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1764,21 +1764,21 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("dense_rank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("lagInFrame", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) @@ -1799,7 +1799,7 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared( name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); } } From 941eba908c406bbfadc90ae8ed01987603512f57 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Aug 2021 21:38:06 +0300 Subject: [PATCH 31/64] Bump librdkafka (to fix metadata cache destroying) This should fix CI under TSan [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/0/9292869c4f92664a28b8c9ddef1e62ddfd13b285/integration_tests_(thread).html Refs: edenhill/librdkafka#3279 --- contrib/librdkafka | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 43491d33ca2..b8554f16820 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 43491d33ca2826531d1e3cae70d4bf1e5249e3c9 +Subproject commit b8554f1682062c85ba519eb54ef2f90e02b812cb From 30dd965e45ed63deffaece4893309934990e68e2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 21:39:06 +0300 Subject: [PATCH 32/64] boop From a7d405759cabc85f7a3a5ada99943102eb32274c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 21:43:43 +0300 Subject: [PATCH 33/64] fix decimal formatting settings in perf test --- docker/test/performance-comparison/compare.sh | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index e5c9f349ce3..c97e8a6ed2b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -628,9 +628,6 @@ cat analyze/errors.log >> report/errors.log ||: cat profile-errors.log >> report/errors.log ||: clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view query_display_names as select * from file('analyze/query-display-names.tsv', TSV, 'test text, query_index int, query_display_name text') @@ -644,6 +641,7 @@ create view partial_query_times as select * from -- Report for partial queries that we could only run on the new server (e.g. -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') + settings output_format_decimal_trailing_zeros = 1 as select toDecimal64(time_median, 3) time, toDecimal64(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name @@ -716,8 +714,9 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') order by test, query_index, metric_name ; -create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as - with +create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion -- of NaN to decimal. @@ -733,8 +732,9 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as changed_fail, test, query_index, query_display_name from queries where changed_show order by abs(diff) desc; -create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') as - select +create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3), toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name from queries where unstable_show order by stat_threshold desc; @@ -764,8 +764,9 @@ create view total_speedup as from test_speedup ; -create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') as - with +create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with (times_speedup >= 1 ? '-' || toString(toDecimal64(times_speedup, 3)) || 'x' : '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x') @@ -791,8 +792,9 @@ create view total_client_time_per_query as select * from file('analyze/client-times.tsv', TSV, 'test text, query_index int, client float, server float'); -create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') as - select client, server, toDecimal64(client/server, 3) p, +create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select client, server, toDecimal64(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) where p > toDecimal64(1.02, 3) order by p desc; @@ -877,8 +879,9 @@ create view test_times_view_total as from test_times_view ; -create table test_times_report engine File(TSV, 'report/test-times.tsv') as - select +create table test_times_report engine File(TSV, 'report/test-times.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select test, toDecimal64(real, 3), toDecimal64(total_client_time, 3), @@ -896,8 +899,9 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as ; -- report for all queries page, only main metric -create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as - with +create table all_tests_report engine File(TSV, 'report/all-queries.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion -- of NaN to decimal. @@ -978,9 +982,6 @@ for version in {right,left} do rm -rf data clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view query_profiles as with 0 as left, 1 as right select * from file('analyze/query-profiles.tsv', TSV, @@ -1063,9 +1064,10 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, ; create table metric_devation engine File(TSVWithNamesAndTypes, - 'report/metric-deviation.$version.tsv') as + 'report/metric-deviation.$version.tsv') + settings output_format_decimal_trailing_zeros = 1 -- first goes the key used to split the file with grep - select test, query_index, query_display_name, + as select test, query_index, query_display_name, toDecimal64(d, 3) d, q, metric from ( select @@ -1176,9 +1178,6 @@ rm -rf metrics ||: mkdir metrics clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view right_async_metric_log as select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes, '$(cat right-async-metric-log.tsv.columns)') @@ -1196,8 +1195,9 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as ; -- Show metrics that have changed -create table changes engine File(TSV, 'metrics/changes.tsv') as - select metric, left, right, +create table changes engine File(TSV, 'metrics/changes.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select metric, left, right, toDecimal64(diff, 3), toDecimal64(times_diff, 3) from ( select metric, median(left) as left, median(right) as right, From 5947e54c1b7982c400c991aebc6bf2723d4e45cb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Aug 2021 21:51:52 +0300 Subject: [PATCH 34/64] Fix incorrect assertion during writing to StorageKafka The problem it does not triggered in CI always because buffers was not destroyed by that time. Fixes: #26547 --- src/Storages/Kafka/WriteBufferToKafkaProducer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index dbb18b56769..34ab48e501d 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -48,7 +48,7 @@ WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() { - assert(rows == 0 && chunks.empty()); + assert(rows == 0); } void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t current_row) From c263523f50ede57e3d8fe36a486723eadf6c91fa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 19 Aug 2021 22:45:40 +0300 Subject: [PATCH 35/64] Remove unused declaration. --- src/Interpreters/Session.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 0e816324dad..d104e250099 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -82,8 +82,6 @@ private: String session_id; std::shared_ptr named_session; bool named_session_created = false; - - static std::optional named_sessions; }; } From 64bfe21a1be96afe31df48c53df4ca408113d776 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 20 Aug 2021 00:25:14 +0300 Subject: [PATCH 36/64] Fix test 00443_preferred_block_size_bytes.sh (#27846) * Update 00443_preferred_block_size_bytes.sh * Update clickhouse-test * Update clickhouse-test * Update database_replicated.xml --- tests/clickhouse-test | 4 +++- tests/config/config.d/database_replicated.xml | 6 +++--- .../queries/0_stateless/00443_preferred_block_size_bytes.sh | 4 ++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c627810a550..0d833e5fbe6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -44,15 +44,17 @@ DISTRIBUTED_DDL_TIMEOUT_MSG = "is executing longer than distributed_ddl_task_tim MESSAGES_TO_RETRY = [ "DB::Exception: ZooKeeper session has been expired", - "DB::Exception: Connection loss", "Coordination::Exception: Session expired", "Coordination::Exception: Connection loss", "Coordination::Exception: Operation timeout", + "DB::Exception: Session expired", + "DB::Exception: Connection loss", "DB::Exception: Operation timeout", "Operation timed out", "ConnectionPoolWithFailover: Connection failed at try", "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", + "DB::Exception: Cannot enqueue query", "Shutdown is called for table", # It happens in SYSTEM SYNC REPLICA query if session with ZooKeeper is being reinitialized. DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME ] diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 9a3b4d68ea6..e51d90dd4d4 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -22,9 +22,9 @@ 10000 30000 1000 - 2000 - 4000 - trace + 4000 + 5000 + information false 1000000000000000 diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 724630057d9..399a4677a44 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -42,10 +42,10 @@ popd > /dev/null #SCRIPTDIR=`dirname "$SCRIPTPATH"` SCRIPTDIR=$SCRIPTPATH -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout 2>&1 +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout 2>&1 +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED rm "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout From 06a0580db14bf7125391615c2b1ec95f67e4da34 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 23:02:01 +0000 Subject: [PATCH 37/64] done --- docker/packager/packager | 1 + src/Common/examples/YAML_fuzzer.cpp | 7 ++++--- src/Parsers/examples/create_parser_fuzzer.cpp | 5 ++++- src/Parsers/examples/select_parser_fuzzer.cpp | 5 ++++- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index c05c85d3e28..00489b40297 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -167,6 +167,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DUSE_GTEST=1') cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DENABLE_EXAMPLES=1') + cmake_flags.append('-DENABLE_FUZZING=1') # Don't stop on first error to find more clang-tidy errors in one run. result.append('NINJA_FLAGS=-k0') diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp index f4b570e0e0a..06e9c34b6cd 100644 --- a/src/Common/examples/YAML_fuzzer.cpp +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -5,6 +5,8 @@ #include #include +#include + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { /// How to test: @@ -18,7 +20,6 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) return 1; } std::string input = std::string(reinterpret_cast(data), size); - DB::YAMLParser parser; { std::ofstream temp_file(file_name); @@ -27,11 +28,11 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) try { - DB::YAMLParser::parse(std::string(file_name)); + DB::YAMLParserImpl::parse(std::string(file_name)); } catch (...) { - std::cerr << "YAML_fuzzer failed: " << getCurrentExceptionMessage() << std::endl; + std::cerr << "YAML_fuzzer failed: " << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << std::endl; return 1; } return 0; diff --git a/src/Parsers/examples/create_parser_fuzzer.cpp b/src/Parsers/examples/create_parser_fuzzer.cpp index 169f7b765b7..b0adec7d634 100644 --- a/src/Parsers/examples/create_parser_fuzzer.cpp +++ b/src/Parsers/examples/create_parser_fuzzer.cpp @@ -15,7 +15,10 @@ try DB::ParserCreateQuery parser; DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - DB::formatAST(*ast, std::cerr); + DB::WriteBufferFromOwnString wb; + DB::formatAST(*ast, wb); + + std::cerr << wb.str() << std::endl; return 0; } diff --git a/src/Parsers/examples/select_parser_fuzzer.cpp b/src/Parsers/examples/select_parser_fuzzer.cpp index 4848b285c07..bdba552dfa7 100644 --- a/src/Parsers/examples/select_parser_fuzzer.cpp +++ b/src/Parsers/examples/select_parser_fuzzer.cpp @@ -14,7 +14,10 @@ try DB::ParserQueryWithOutput parser(input.data() + input.size()); DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - DB::formatAST(*ast, std::cerr); + DB::WriteBufferFromOwnString wb; + DB::formatAST(*ast, wb); + + std::cerr << wb.str() << std::endl; return 0; } From d575f06dffe7d15d8e2145a852c4526a6e29ac4e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 23:04:55 +0000 Subject: [PATCH 38/64] better --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 00489b40297..ba78d1df583 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -167,7 +167,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DUSE_GTEST=1') cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DENABLE_EXAMPLES=1') - cmake_flags.append('-DENABLE_FUZZING=1') + cmake_flags.append('-DENABLE_FUZZING=1') # Don't stop on first error to find more clang-tidy errors in one run. result.append('NINJA_FLAGS=-k0') From f5af6fae2f953231959f9f02a8d5e5fd01155a47 Mon Sep 17 00:00:00 2001 From: igomac <714541080@qq.com> Date: Fri, 20 Aug 2021 12:03:49 +0800 Subject: [PATCH 39/64] Update docs/zh/sql-reference/functions/ym-dict-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/zh/sql-reference/functions/ym-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index fb2c31291ad..86bf4fbe2fb 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -135,7 +135,7 @@ regionToTopContinent(id[, geobase]) ### regionToName(id\[, lang\]) {#regiontonameid-lang} -从 Yandex gebase 接收一个 UInt32 数字类型的 region ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数,则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在,则返回一个空字符串。 +从 Yandex geobase 接收一个 UInt32 数字类型的 region ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数,则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在,则返回一个空字符串。 `ua` 和 `uk` 都意味着乌克兰。 From 0a5c979907b6c7b40400b32209a6655abe31b278 Mon Sep 17 00:00:00 2001 From: igomac <714541080@qq.com> Date: Fri, 20 Aug 2021 12:03:57 +0800 Subject: [PATCH 40/64] Update docs/zh/sql-reference/functions/ym-dict-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/zh/sql-reference/functions/ym-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index 86bf4fbe2fb..948139aff29 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -130,7 +130,7 @@ regionToTopContinent(id[, geobase]) ### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase} -从 Yandex gebase 接收一个 UInt32 数字类型的 region ID 。返回一个区域id数组,由传递的区域和链上的所有父节点组成。 +从 Yandex geobase 接收一个 UInt32 数字类型的 region ID 。返回一个区域id数组,由传递的区域和链上的所有父节点组成。 示例: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. ### regionToName(id\[, lang\]) {#regiontonameid-lang} From b60e5ac801ebfe025e321b33bd48dd26648a94d0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 20 Aug 2021 07:14:20 +0300 Subject: [PATCH 41/64] Fix incorrect assertion during writing to StorageRabbitMQ --- src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index b9af60eb66f..be7f1fe508a 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -101,7 +101,7 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - assert(rows == 0 && chunks.empty()); + assert(rows == 0); } From 1d7c58238f6d0c130e6246ec0684233de9f5e614 Mon Sep 17 00:00:00 2001 From: igomac <714541080@qq.com> Date: Fri, 20 Aug 2021 12:27:19 +0800 Subject: [PATCH 42/64] Create ym-dict-functions.md add missing translation --- .../functions/ym-dict-functions.md | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index 948139aff29..0501d6e82a5 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -20,13 +20,13 @@ ClickHouse支持同时使用多个备选地理基(区域层次结构),以 所有处理区域的函数都在末尾有一个可选参数—字典键。它被称为地基。 示例: - regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txt - regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txt - regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt + regionToCountry(RegionID) – 使用默认路径: /opt/geo/regions_hierarchy.txt + regionToCountry(RegionID, '') – 使用默认路径: /opt/geo/regions_hierarchy.txt + regionToCountry(RegionID, 'ua') – 使用字典中的'ua' 键: /opt/geo/regions_hierarchy_ua.txt ### regionToCity(id[, geobase]) {#regiontocityid-geobase} -从 Yandex gebase 接收一个 UInt32 数字类型的 region ID 。如果该区域是一个城市或城市的一部分,它将返回相应城市的区域ID。否则,返回0。 +从 Yandex gebase 接收一个 UInt32 数字类型的区域ID 。如果该区域是一个城市或城市的一部分,它将返回相应城市的区域ID。否则,返回0。 ### regionToArea(id[, geobase]) {#regiontoareaid-geobase} @@ -106,13 +106,13 @@ regionToTopContinent(id[, geobase]) **参数** -- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Yandex geobase 的区域 ID. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — 字典的建. 参阅 [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). 可选. **返回值** -- Identifier of the top level continent (the latter when you climb the hierarchy of regions). -- 0, if there is none. +- 顶级大陆的标识符(当您在区域层次结构中攀爬时,是后者)。 +- 0,如果没有。 类型: `UInt32`. @@ -130,12 +130,12 @@ regionToTopContinent(id[, geobase]) ### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase} -从 Yandex geobase 接收一个 UInt32 数字类型的 region ID 。返回一个区域id数组,由传递的区域和链上的所有父节点组成。 +从 Yandex geobase 接收一个 UInt32 数字类型的区域ID。返回一个区域ID数组,由传递的区域和链上的所有父节点组成。 示例: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. ### regionToName(id\[, lang\]) {#regiontonameid-lang} -从 Yandex geobase 接收一个 UInt32 数字类型的 region ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数,则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在,则返回一个空字符串。 +从 Yandex geobase 接收一个 UInt32 数字类型的区域ID。带有语言名称的字符串可以作为第二个参数传递。支持的语言有:ru, en, ua, uk, by, kz, tr。如果省略第二个参数,则使用' ru '语言。如果不支持该语言,则抛出异常。返回一个字符串-对应语言的区域名称。如果指定ID的区域不存在,则返回一个空字符串。 `ua` 和 `uk` 都意味着乌克兰。 From 2ab97bd621304e527cd2cafcf03b851d7302f630 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 20 Aug 2021 11:08:21 +0300 Subject: [PATCH 43/64] Improve usability when user forgot password #27750 --- programs/client/Client.cpp | 56 ++++++++++++++++++++++++++------------ 1 file changed, 38 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 50751de43a4..65e245750b3 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -129,6 +129,7 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int TOO_DEEP_RECURSION; + extern const int AUTHENTICATION_FAILED; } @@ -773,31 +774,50 @@ private: << connection_parameters.host << ":" << connection_parameters.port << (!connection_parameters.user.empty() ? " as user " + connection_parameters.user : "") << "." << std::endl; - connection = std::make_unique( - connection_parameters.host, - connection_parameters.port, - connection_parameters.default_database, - connection_parameters.user, - connection_parameters.password, - "", /* cluster */ - "", /* cluster_secret */ - "client", - connection_parameters.compression, - connection_parameters.security); - String server_name; UInt64 server_version_major = 0; UInt64 server_version_minor = 0; UInt64 server_version_patch = 0; - if (max_client_network_bandwidth) + try { - ThrottlerPtr throttler = std::make_shared(max_client_network_bandwidth, 0, ""); - connection->setThrottler(throttler); - } + connection = std::make_unique( + connection_parameters.host, + connection_parameters.port, + connection_parameters.default_database, + connection_parameters.user, + connection_parameters.password, + "", /* cluster */ + "", /* cluster_secret */ + "client", + connection_parameters.compression, + connection_parameters.security); - connection->getServerVersion( - connection_parameters.timeouts, server_name, server_version_major, server_version_minor, server_version_patch, server_revision); + if (max_client_network_bandwidth) + { + ThrottlerPtr throttler = std::make_shared(max_client_network_bandwidth, 0, ""); + connection->setThrottler(throttler); + } + + connection->getServerVersion( + connection_parameters.timeouts, server_name, server_version_major, server_version_minor, server_version_patch, server_revision); + } + catch (const Exception & e) + { + /// It is typical when users install ClickHouse, type some password and instantly forget it. + if ((connection_parameters.user.empty() || connection_parameters.user == "default") + && e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED) + { + std::cerr << std::endl + << "If you have installed ClickHouse and forgot password you can reset it in the configuration file." << std::endl + << "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml" << std::endl + << "and deleting this file will reset the password." << std::endl + << "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed." << std::endl + << std::endl; + } + + throw; + } server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); From 9831995fd85cd638a1e809875a9db699e8b64c06 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 20 Aug 2021 12:09:05 +0300 Subject: [PATCH 44/64] Dictionary empty attribute list --- src/Dictionaries/DictionaryStructure.cpp | 3 --- .../getDictionaryConfigurationFromAST.cpp | 5 ----- .../02011_dictionary_empty_attribute_list.reference | 3 +++ .../02011_dictionary_empty_attribute_list.sql | 12 ++++++++++++ 4 files changed, 15 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02011_dictionary_empty_attribute_list.reference create mode 100644 tests/queries/0_stateless/02011_dictionary_empty_attribute_list.sql diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 9f46addd912..15cc75fab19 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -192,9 +192,6 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration } } - if (attributes.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary has no attributes defined"); - if (config.getBool(config_prefix + ".layout.ip_trie.access_to_key_from_attributes", false)) access_to_key_from_attributes = true; } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index ba81b1f1364..c8cb2611651 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -496,9 +496,6 @@ void checkAST(const ASTCreateQuery & query) if (!query.is_dictionary || query.dictionary == nullptr) throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot convert dictionary to configuration from non-dictionary AST."); - if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty()) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty attributes list"); - if (query.dictionary->layout == nullptr) throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty layout"); @@ -512,8 +509,6 @@ void checkAST(const ASTCreateQuery & query) if (query.dictionary->source == nullptr) throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty source"); - - /// Range can be empty } void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs) diff --git a/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.reference b/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.reference new file mode 100644 index 00000000000..7938dcdde86 --- /dev/null +++ b/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.reference @@ -0,0 +1,3 @@ +0 +1 +0 diff --git a/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.sql b/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.sql new file mode 100644 index 00000000000..5c0668cb839 --- /dev/null +++ b/tests/queries/0_stateless/02011_dictionary_empty_attribute_list.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table (id UInt64) ENGINE=TinyLog; +INSERT INTO test_table VALUES (0); + +DROP DICTIONARY IF EXISTS test_dictionary; +CREATE DICTIONARY test_dictionary (id UInt64) PRIMARY KEY id LAYOUT(DIRECT()) SOURCE(CLICKHOUSE(TABLE 'test_table')); +SELECT * FROM test_dictionary; +SELECT dictHas('test_dictionary', toUInt64(0)); +SELECT dictHas('test_dictionary', toUInt64(1)); + +DROP DICTIONARY test_dictionary; +DROP TABLE test_table; From c6f456b24162997c63014334e704cc7d7d43dac4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 20 Aug 2021 09:08:39 +0000 Subject: [PATCH 45/64] Fix build --- src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++------ src/Interpreters/TreeOptimizer.cpp | 2 +- src/Parsers/ASTSelectQuery.h | 1 + 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 94dadddcf13..7692eb4491e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -168,7 +168,7 @@ static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * se /// Case when GROUP BY element is position. /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are constants. - if (auto * ast_literal = typeid_cast(argument.get())) + if (const auto * ast_literal = typeid_cast(argument.get())) { auto which = ast_literal->value.getType(); if (which == Field::Types::UInt64) @@ -1326,17 +1326,16 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain } auto & children = select_query->limitBy()->children; - - for (size_t i = 0; i < children.size(); ++i) + for (auto & child : children) { if (getContext()->getSettingsRef().enable_positional_arguments) { - auto new_argument = checkPositionalArgument(children[i], select_query, ASTSelectQuery::Expression::LIMIT_BY); + auto new_argument = checkPositionalArgument(child, select_query, ASTSelectQuery::Expression::LIMIT_BY); if (new_argument) - children[i] = new_argument; + child = new_argument; } - auto child_name = children[i]->getColumnName(); + auto child_name = child->getColumnName(); if (!aggregated_names.count(child_name)) step.addRequiredOutput(std::move(child_name)); } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 1d2b0670f8c..518c041d785 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -173,7 +173,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum bool keep_position = false; if (settings.enable_positional_arguments) { - const auto & value = group_exprs[i]->as ()->value; + const auto & value = group_exprs[i]->as()->value; if (value.getType() == Field::Types::UInt64) { auto pos = value.get(); diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index e439c5edaa5..2babc2f75c8 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -70,6 +70,7 @@ public: case Expression::SETTINGS: return "SETTINGS"; } + return ""; } /** Get the text that identifies this element. */ From 0aa800122d6b058513ad9d19f964f0caca02bc28 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 20 Aug 2021 12:28:23 +0300 Subject: [PATCH 46/64] Update menus.md --- docs/en/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 87e4c75d0d4..8f330f39226 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -105,7 +105,7 @@ We use `Decimal` data type to store prices. Everything else is quite straightfor ## Import Data -Upload data into ClickHouse in parallel: +Upload data into ClickHouse: ``` clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv From c3c31e2895ee443b3503e573aaba39a3ae29ca65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 20 Aug 2021 12:56:39 +0300 Subject: [PATCH 47/64] Move function to appropriate place to make code more readable --- src/Server/TCPHandler.cpp | 48 +++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b2db65e22bc..beca726e95f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -49,27 +49,6 @@ namespace DB { -namespace -{ -std::string formatHTTPErrorResponse(const Poco::Util::AbstractConfiguration& config) -{ - std::string result = fmt::format( - "HTTP/1.0 400 Bad Request\r\n\r\n" - "Port {} is for clickhouse-client program\r\n", - config.getString("tcp_port")); - - if (config.has("http_port")) - { - result += fmt::format( - "You must use port {} for HTTP.\r\n", - config.getString("http_port")); - } - - return result; -} -} - - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -925,6 +904,29 @@ bool TCPHandler::receiveProxyHeader() } +namespace +{ + +std::string formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(const Poco::Util::AbstractConfiguration& config) +{ + std::string result = fmt::format( + "HTTP/1.0 400 Bad Request\r\n\r\n" + "Port {} is for clickhouse-client program\r\n", + config.getString("tcp_port")); + + if (config.has("http_port")) + { + result += fmt::format( + "You must use port {} for HTTP.\r\n", + config.getString("http_port")); + } + + return result; +} + +} + + void TCPHandler::receiveHello() { /// Receive `hello` packet. @@ -940,9 +942,7 @@ void TCPHandler::receiveHello() */ if (packet_type == 'G' || packet_type == 'P') { - writeString(formatHTTPErrorResponse(server.config()), - *out); - + writeString(formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(server.config()), *out); throw Exception("Client has connected to wrong port", ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT); } else From cbc1b2e72d784df275a4ffa964393b6cc0cda39e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 20 Aug 2021 13:10:12 +0300 Subject: [PATCH 48/64] Fix selecting with extremes from LowCardinality(UUID) --- src/Columns/ColumnUnique.h | 2 +- ...02012_low_cardinality_uuid_with_extremes.reference | 4 ++++ .../02012_low_cardinality_uuid_with_extremes.sql | 11 +++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.reference create mode 100644 tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.sql diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 8ca4baff7c7..bfa80b5e3b2 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -304,7 +304,7 @@ size_t ColumnUnique::uniqueInsert(const Field & x) if (x.getType() == Field::Types::Null) return getNullValueIndex(); - if (isNumeric()) + if (valuesHaveFixedSize()) return uniqueInsertData(&x.reinterpret(), size_of_value_if_fixed); auto & val = x.get(); diff --git a/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.reference b/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.reference new file mode 100644 index 00000000000..af2447df807 --- /dev/null +++ b/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.reference @@ -0,0 +1,4 @@ +0562380c-d1f3-4091-83d5-8c972f534317 + +0562380c-d1f3-4091-83d5-8c972f534317 +0562380c-d1f3-4091-83d5-8c972f534317 diff --git a/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.sql b/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.sql new file mode 100644 index 00000000000..191383cc978 --- /dev/null +++ b/tests/queries/0_stateless/02012_low_cardinality_uuid_with_extremes.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS tbl; + +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE tbl (`lc` LowCardinality(UUID)) ENGINE = Memory; + +INSERT INTO tbl VALUES ('0562380c-d1f3-4091-83d5-8c972f534317'); + +SET extremes = 1; +SELECT * FROM tbl; + +DROP TABLE tbl; From 0da15b9ece64c78e6c5cd062a5dc2e14ba469c3d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 20 Aug 2021 10:23:50 +0000 Subject: [PATCH 49/64] Fix build --- docker/packager/packager | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/packager b/docker/packager/packager index ba78d1df583..2a7f1d3631a 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -168,6 +168,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DENABLE_EXAMPLES=1') cmake_flags.append('-DENABLE_FUZZING=1') + # For fuzzing needs + cmake_flags.append('-DUSE_YAML_CPP=1') # Don't stop on first error to find more clang-tidy errors in one run. result.append('NINJA_FLAGS=-k0') From 8c6dd189178b7a473f8e3d963dd17087a6458537 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Aug 2021 14:55:04 +0300 Subject: [PATCH 50/64] check cluster name before creating Distributed --- src/Interpreters/Context.cpp | 4 ++-- src/Storages/StorageDistributed.cpp | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a634c19dcd6..3058132dc36 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1821,8 +1821,8 @@ std::shared_ptr Context::getCluster(const std::string & cluster_name) c auto res = getClusters()->getCluster(cluster_name); if (res) return res; - - res = tryGetReplicatedDatabaseCluster(cluster_name); + if (!cluster_name.empty()) + res = tryGetReplicatedDatabaseCluster(cluster_name); if (res) return res; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fcd0e255e5c..15355c997ff 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -363,10 +363,13 @@ StorageDistributed::StorageDistributed( } /// Sanity check. Skip check if the table is already created to allow the server to start. - if (!attach_ && !cluster_name.empty()) + if (!attach_) { - size_t num_local_shards = getContext()->getCluster(cluster_name)->getLocalShardCount(); - if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name) + if (remote_database.empty() && !remote_table_function_ptr && !getCluster()->maybeCrossReplication()) + LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly."); + + size_t num_local_shards = getCluster()->getLocalShardCount(); + if (num_local_shards && (remote_database.empty() || remote_database == id_.database_name) && remote_table == id_.table_name) throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP); } } @@ -810,9 +813,6 @@ void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_co void StorageDistributed::startup() { - if (remote_database.empty() && !remote_table_function_ptr && !getCluster()->maybeCrossReplication()) - LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly."); - if (!storage_policy) return; From c6f94e50163764ba8656f6a52956f40ed7b5fe73 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 20 Aug 2021 11:57:15 +0000 Subject: [PATCH 51/64] better --- docker/packager/packager | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index 2a7f1d3631a..d0b604c16c2 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -168,8 +168,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DENABLE_EXAMPLES=1') cmake_flags.append('-DENABLE_FUZZING=1') - # For fuzzing needs - cmake_flags.append('-DUSE_YAML_CPP=1') + # For fuzzing needs + cmake_flags.append('-DUSE_YAML_CPP=1') # Don't stop on first error to find more clang-tidy errors in one run. result.append('NINJA_FLAGS=-k0') From 452602dadb460fa057f2504a898d9f1bd2ce47eb Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 20 Aug 2021 14:44:51 +0800 Subject: [PATCH 52/64] APPLY with lambda. --- src/Parsers/ASTColumnsTransformers.cpp | 45 +++++++++++++-- src/Parsers/ASTColumnsTransformers.h | 11 +++- src/Parsers/ExpressionElementParsers.cpp | 57 +++++++++++++++---- .../01470_columns_transformers2.reference | 3 + .../01470_columns_transformers2.sql | 3 + 5 files changed, 101 insertions(+), 18 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 359efbd03aa..451ecf0d4dd 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -40,10 +41,18 @@ void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, For if (!column_name_prefix.empty()) settings.ostr << "("; - settings.ostr << func_name; - if (parameters) - parameters->formatImpl(settings, state, frame); + if (lambda) + { + lambda->formatImpl(settings, state, frame); + } + else + { + settings.ostr << func_name; + + if (parameters) + parameters->formatImpl(settings, state, frame); + } if (!column_name_prefix.empty()) settings.ostr << ", '" << column_name_prefix << "')"; @@ -64,9 +73,33 @@ void ASTColumnsApplyTransformer::transform(ASTs & nodes) const else name = column->getColumnName(); } - auto function = makeASTFunction(func_name, column); - function->parameters = parameters; - column = function; + if (lambda) + { + auto body = lambda->as().arguments->children.at(1)->clone(); + std::stack stack; + stack.push(body); + while (!stack.empty()) + { + auto ast = stack.top(); + stack.pop(); + for (auto & child : ast->children) + { + if (auto arg_name = tryGetIdentifierName(child); arg_name && arg_name == lambda_arg) + { + child = column->clone(); + continue; + } + stack.push(child); + } + } + column = body; + } + else + { + auto function = makeASTFunction(func_name, column); + function->parameters = parameters; + column = function; + } if (!column_name_prefix.empty()) column->setAlias(column_name_prefix + name); } diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index 49d29222f02..1064beb44bd 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -25,13 +25,22 @@ public: auto res = std::make_shared(*this); if (parameters) res->parameters = parameters->clone(); + if (lambda) + res->lambda = lambda->clone(); return res; } void transform(ASTs & nodes) const override; + + // Case 1 APPLY (quantile(0.9)) String func_name; - String column_name_prefix; ASTPtr parameters; + // Case 2 APPLY (x -> quantile(0.9)(x)) + ASTPtr lambda; + String lambda_arg; + + String column_name_prefix; + protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 16f2b720b4a..bf584551570 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1811,20 +1811,47 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e with_open_round_bracket = true; } + ASTPtr lambda; + String lambda_arg; ASTPtr func_name; - if (!ParserIdentifier().parse(pos, func_name, expected)) - return false; - ASTPtr expr_list_args; - if (pos->type == TokenType::OpeningRoundBracket) + auto opos = pos; + if (ParserLambdaExpression().parse(pos, lambda, expected)) { - ++pos; - if (!ParserExpressionList(false).parse(pos, expr_list_args, expected)) + if (const auto * func = lambda->as(); func && func->name == "lambda") + { + const auto * lambda_args_tuple = func->arguments->children.at(0)->as(); + const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; + if (lambda_arg_asts.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "APPLY column transformer can only accept lambda with one argument"); + + if (auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[0]); opt_arg_name) + lambda_arg = *opt_arg_name; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "lambda argument declarations must be identifiers"); + } + else + { + lambda = nullptr; + pos = opos; + } + } + + if (!lambda) + { + if (!ParserIdentifier().parse(pos, func_name, expected)) return false; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; + if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; + if (!ParserExpressionList(false).parse(pos, expr_list_args, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } } String column_name_prefix; @@ -1848,8 +1875,16 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e } auto res = std::make_shared(); - res->func_name = getIdentifierName(func_name); - res->parameters = expr_list_args; + if (lambda) + { + res->lambda = lambda; + res->lambda_arg = lambda_arg; + } + else + { + res->func_name = getIdentifierName(func_name); + res->parameters = expr_list_args; + } res->column_name_prefix = column_name_prefix; node = std::move(res); return true; diff --git a/tests/queries/0_stateless/01470_columns_transformers2.reference b/tests/queries/0_stateless/01470_columns_transformers2.reference index 18c0f5c7e89..6d196530135 100644 --- a/tests/queries/0_stateless/01470_columns_transformers2.reference +++ b/tests/queries/0_stateless/01470_columns_transformers2.reference @@ -1 +1,4 @@ 100 10 324 120.00 B 8.00 B 23.00 B +0 +SELECT argMax(number, number) +FROM numbers(1) diff --git a/tests/queries/0_stateless/01470_columns_transformers2.sql b/tests/queries/0_stateless/01470_columns_transformers2.sql index 3691ef1e65d..88513d023c4 100644 --- a/tests/queries/0_stateless/01470_columns_transformers2.sql +++ b/tests/queries/0_stateless/01470_columns_transformers2.sql @@ -5,3 +5,6 @@ INSERT INTO columns_transformers VALUES (100, 10, 324, 120, 8, 23); SELECT * EXCEPT 'bytes', COLUMNS('bytes') APPLY formatReadableSize FROM columns_transformers; DROP TABLE IF EXISTS columns_transformers; + +SELECT * APPLY x->argMax(x, number) FROM numbers(1); +EXPLAIN SYNTAX SELECT * APPLY x->argMax(x, number) FROM numbers(1); From c5779eb604ec68aca944990723cb9247d8491cdc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 20 Aug 2021 13:27:44 +0000 Subject: [PATCH 53/64] Fix build --- docker/packager/packager | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index d0b604c16c2..b51e254848f 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -168,8 +168,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DENABLE_EXAMPLES=1') cmake_flags.append('-DENABLE_FUZZING=1') - # For fuzzing needs - cmake_flags.append('-DUSE_YAML_CPP=1') + # For fuzzing needs + cmake_flags.append('-DUSE_YAML_CPP=1') # Don't stop on first error to find more clang-tidy errors in one run. result.append('NINJA_FLAGS=-k0') From 42378b5913178e4ffe3702266594a4c58d336b12 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Aug 2021 17:05:53 +0300 Subject: [PATCH 54/64] fix --- src/Storages/StorageDistributed.cpp | 8 +++++--- src/Storages/StorageDistributed.h | 3 ++- .../0_stateless/00987_distributed_stack_overflow.sql | 4 +--- tests/queries/0_stateless/01763_max_distributed_depth.sql | 4 +++- 4 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 15355c997ff..df7d568deb9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -327,11 +327,13 @@ StorageDistributed::StorageDistributed( const String & relative_data_path_, const DistributedSettings & distributed_settings_, bool attach_, - ClusterPtr owned_cluster_) + ClusterPtr owned_cluster_, + ASTPtr remote_table_function_ptr_) : IStorage(id_) , WithContext(context_->getGlobalContext()) , remote_database(remote_database_) , remote_table(remote_table_) + , remote_table_function_ptr(remote_table_function_ptr_) , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) , owned_cluster(std::move(owned_cluster_)) , cluster_name(getContext()->getMacros()->expand(cluster_name_)) @@ -402,9 +404,9 @@ StorageDistributed::StorageDistributed( relative_data_path_, distributed_settings_, attach, - std::move(owned_cluster_)) + std::move(owned_cluster_), + remote_table_function_ptr_) { - remote_table_function_ptr = std::move(remote_table_function_ptr_); } QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index b6a26467a3f..b003f8c6486 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -136,7 +136,8 @@ private: const String & relative_data_path_, const DistributedSettings & distributed_settings_, bool attach_, - ClusterPtr owned_cluster_ = {}); + ClusterPtr owned_cluster_ = {}, + ASTPtr remote_table_function_ptr_ = {}); StorageDistributed( const StorageID & id_, diff --git a/tests/queries/0_stateless/00987_distributed_stack_overflow.sql b/tests/queries/0_stateless/00987_distributed_stack_overflow.sql index d2e2b8f37ef..1ef7c543252 100644 --- a/tests/queries/0_stateless/00987_distributed_stack_overflow.sql +++ b/tests/queries/0_stateless/00987_distributed_stack_overflow.sql @@ -4,8 +4,7 @@ DROP TABLE IF EXISTS distr2; CREATE TABLE distr (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr); -- { serverError 269 } -CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); -SELECT * FROM distr0; -- { serverError 581 } +CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); -- { serverError 269 } CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2); CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1); @@ -13,6 +12,5 @@ CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, current SELECT * FROM distr1; -- { serverError 581 } SELECT * FROM distr2; -- { serverError 581 } -DROP TABLE distr0; DROP TABLE distr1; DROP TABLE distr2; diff --git a/tests/queries/0_stateless/01763_max_distributed_depth.sql b/tests/queries/0_stateless/01763_max_distributed_depth.sql index d1bb9e4be90..89909a3bd8d 100644 --- a/tests/queries/0_stateless/01763_max_distributed_depth.sql +++ b/tests/queries/0_stateless/01763_max_distributed_depth.sql @@ -9,7 +9,9 @@ CREATE TABLE tt6 `status` String ) -ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand()); +ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand()); + +CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand()); INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 } From 5708ed42ec8bb5ffe939ee0824fea036ee9855cd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 20 Aug 2021 22:27:47 +0800 Subject: [PATCH 55/64] Add setting. empty_result_for_aggregation_by_constant_keys_on_empty_set --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 4 +++- .../01925_test_group_by_const_consistency.reference | 1 + .../0_stateless/01925_test_group_by_const_consistency.sql | 6 ++++-- 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..7f86520c573 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -249,6 +249,7 @@ class IColumn; M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ + M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 33f9deaf805..20ab3152087 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2044,7 +2044,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes, settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set || (keys.empty() && query_analyzer->hasConstAggregationKeys()), + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, diff --git a/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference b/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference index 573541ac970..93f9e3d10db 100644 --- a/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference +++ b/tests/queries/0_stateless/01925_test_group_by_const_consistency.reference @@ -1 +1,2 @@ 0 +1 0 diff --git a/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql b/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql index 8a5de0e7c4f..a73c06bbe49 100644 --- a/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql +++ b/tests/queries/0_stateless/01925_test_group_by_const_consistency.sql @@ -1,2 +1,4 @@ -SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a; -SELECT count() FROM numbers(10) WHERE 0 +SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a; +SELECT count() FROM numbers(10) WHERE 0; + +SELECT 1 as a, count() FROM numbers(10) WHERE 0 GROUP BY a SETTINGS empty_result_for_aggregation_by_constant_keys_on_empty_set = 0; From 812a6ffb80780637876caa160607cb708893919f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 20 Aug 2021 18:56:15 +0300 Subject: [PATCH 56/64] kafka assertion was fixed in master --- tests/integration/test_storage_kafka/test.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 850112144f8..ed98d96bdd1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -672,9 +672,6 @@ def describe_consumer_group(kafka_cluster, name): def kafka_cluster(): try: cluster.start() - if instance.is_debug_build(): - # https://github.com/ClickHouse/ClickHouse/issues/26547 - pytest.skip("~WriteBufferToKafkaProducer(): Assertion `rows == 0 && chunks.empty()' failed.") kafka_id = instance.cluster.kafka_docker_id print(("kafka_id is {}".format(kafka_id))) yield cluster From 58c1b57259478661e26e29e7bf505f9aaab9b452 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 20 Aug 2021 18:58:43 +0300 Subject: [PATCH 57/64] improve tests from test_backward_compatibility --- tests/integration/helpers/cluster.py | 34 +++++++++++++++++++ .../test_aggregate_function_state_avg.py | 6 ++++ .../test_data_skipping_indices.py | 4 +-- .../test_detach_part_wrong_partition_id.py | 4 +++ .../test_select_aggregate_alias_column.py | 3 ++ .../test_short_strings_aggregation.py | 2 ++ 6 files changed, 51 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c6b44a8b830..ea269de3a9e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2028,6 +2028,37 @@ class ClickHouseInstance: return None return None + def restart_with_original_version(self, stop_start_wait_sec=300, callback_onstop=None, signal=15): + if not self.stay_alive: + raise Exception("Cannot restart not stay alive container") + self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') + retries = int(stop_start_wait_sec / 0.5) + local_counter = 0 + # wait stop + while local_counter < retries: + if not self.get_process_pid("clickhouse server"): + break + time.sleep(0.5) + local_counter += 1 + + # force kill if server hangs + if self.get_process_pid("clickhouse server"): + # server can die before kill, so don't throw exception, it's expected + self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(9)], nothrow=True, user='root') + + if callback_onstop: + callback_onstop(self) + self.exec_in_container( + ["bash", "-c", "cp /usr/share/clickhouse_original /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], + user='root') + self.exec_in_container(["bash", "-c", + "cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"], + user='root') + self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid())) + + # wait start + assert_eq_with_retry(self, "select 1", "1", retry_count=retries) + def restart_with_latest_version(self, stop_start_wait_sec=300, callback_onstop=None, signal=15): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") @@ -2048,6 +2079,9 @@ class ClickHouseInstance: if callback_onstop: callback_onstop(self) + self.exec_in_container( + ["bash", "-c", "cp /usr/bin/clickhouse /usr/share/clickhouse_original"], + user='root') self.exec_in_container( ["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], user='root') diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py index 5ed97e7a9a5..feaf96c439d 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py @@ -53,3 +53,9 @@ def test_backward_compatability(start_cluster): node1.restart_with_latest_version() assert (node1.query("SELECT avgMerge(x) FROM state") == '2.5\n') + + node1.query("drop table tab") + node1.query("drop table state") + node2.query("drop table tab") + node3.query("drop table tab") + node4.query("drop table tab") \ No newline at end of file diff --git a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py index 45b85897798..db6a3eb7a08 100644 --- a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py +++ b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py @@ -5,7 +5,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="skipping_indices") node = cluster.add_instance('node', image='yandex/clickhouse-server', tag='21.6', stay_alive=True, with_installed_binary=True) @@ -41,4 +41,4 @@ def test_index(start_cluster): node.query(""" SELECT * FROM data WHERE value = 20000 SETTINGS force_data_skipping_indices = 'value_index' SETTINGS force_data_skipping_indices = 'value_index', max_rows_to_read=1; DROP TABLE data; - """) + """) \ No newline at end of file diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 7c20b3c2476..abebaaea8b8 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -30,3 +30,7 @@ def test_detach_part_wrong_partition_id(start_cluster): num_detached = node_21_6.query("select count() from system.detached_parts") assert num_detached == '1\n' + + node_21_6.restart_with_original_version() + + node_21_6.query("drop table tab SYNC") diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index b3f5c68cf68..9a7c7f73eb5 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -27,3 +27,6 @@ def test_select_aggregate_alias_column(start_cluster): node1.query("select sum(x_alias) from remote('node{1,2}', default, tab)") node2.query("select sum(x_alias) from remote('node{1,2}', default, tab)") + + node1.query("drop table tab") + node2.query("drop table tab") \ No newline at end of file diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 463fadc36e8..54dd53c344e 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -29,3 +29,5 @@ def test_backward_compatability(start_cluster): "select s, count() from remote('node{1,2}', default, tab) group by s order by toUInt64(s) limit 50") print(res) assert res == ''.join('{}\t2\n'.format(i) for i in range(50)) + node1.query("drop table tab") + node2.query("drop table tab") From 91813b01106b08b8cee08d9c88dcf13ec9b0df66 Mon Sep 17 00:00:00 2001 From: igomac <714541080@qq.com> Date: Sat, 21 Aug 2021 00:53:19 +0800 Subject: [PATCH 58/64] Update docs/zh/sql-reference/functions/ym-dict-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/zh/sql-reference/functions/ym-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/functions/ym-dict-functions.md b/docs/zh/sql-reference/functions/ym-dict-functions.md index 0501d6e82a5..2f2b2f80d25 100644 --- a/docs/zh/sql-reference/functions/ym-dict-functions.md +++ b/docs/zh/sql-reference/functions/ym-dict-functions.md @@ -26,7 +26,7 @@ ClickHouse支持同时使用多个备选地理基(区域层次结构),以 ### regionToCity(id[, geobase]) {#regiontocityid-geobase} -从 Yandex gebase 接收一个 UInt32 数字类型的区域ID 。如果该区域是一个城市或城市的一部分,它将返回相应城市的区域ID。否则,返回0。 +从 Yandex geobase 接收一个 UInt32 数字类型的区域ID 。如果该区域是一个城市或城市的一部分,它将返回相应城市的区域ID。否则,返回0。 ### regionToArea(id[, geobase]) {#regiontoareaid-geobase} From 0abcc5b18a719148c7e40483a68ced5d4ee8384c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 20 Aug 2021 21:27:34 +0000 Subject: [PATCH 59/64] Fix build --- src/Common/examples/YAML_fuzzer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp index 06e9c34b6cd..556ed13bb04 100644 --- a/src/Common/examples/YAML_fuzzer.cpp +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -28,7 +28,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) try { - DB::YAMLParserImpl::parse(std::string(file_name)); + DB::YAMLParser::parse(std::string(file_name)); } catch (...) { @@ -37,4 +37,3 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) } return 0; } - From f72457fa98f1c5726ca3081ae736aaf56061ee0c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 02:20:23 +0000 Subject: [PATCH 60/64] Lower the size of the binaries --- src/Common/examples/CMakeLists.txt | 87 ------- src/Common/examples/YAML_fuzzer.cpp | 39 ---- src/Compression/CMakeLists.txt | 13 +- src/Compression/CompressionFactory.cpp | 180 --------------- src/Compression/CompressionFactory.h | 2 + .../CompressionFactoryAdditions.cpp | 212 ++++++++++++++++++ src/DataTypes/IDataType.cpp | 17 -- src/DataTypes/IDataType.h | 10 +- src/Storages/CMakeLists.txt | 6 +- src/Storages/examples/CMakeLists.txt | 7 - src/Storages/fuzzers/CMakeLists.txt | 11 + .../columns_description_fuzzer.cpp | 0 .../mergetree_checksum_fuzzer.cpp | 0 13 files changed, 250 insertions(+), 334 deletions(-) delete mode 100644 src/Common/examples/CMakeLists.txt delete mode 100644 src/Common/examples/YAML_fuzzer.cpp create mode 100644 src/Compression/CompressionFactoryAdditions.cpp create mode 100644 src/Storages/fuzzers/CMakeLists.txt rename src/Storages/{examples => fuzzers}/columns_description_fuzzer.cpp (100%) rename src/Storages/{examples => fuzzers}/mergetree_checksum_fuzzer.cpp (100%) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt deleted file mode 100644 index b154d5965ca..00000000000 --- a/src/Common/examples/CMakeLists.txt +++ /dev/null @@ -1,87 +0,0 @@ -add_executable (hashes_test hashes_test.cpp) -target_link_libraries (hashes_test PRIVATE clickhouse_common_io ${CITYHASH_LIBRARIES}) -if(OPENSSL_CRYPTO_LIBRARY) - target_link_libraries (hashes_test PRIVATE ${OPENSSL_CRYPTO_LIBRARY}) -endif() - -add_executable (sip_hash_perf sip_hash_perf.cpp) -target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) - -add_executable (small_table small_table.cpp) -target_link_libraries (small_table PRIVATE clickhouse_common_io) - -add_executable (parallel_aggregation parallel_aggregation.cpp) -target_link_libraries (parallel_aggregation PRIVATE dbms) - -add_executable (parallel_aggregation2 parallel_aggregation2.cpp) -target_link_libraries (parallel_aggregation2 PRIVATE dbms) - -add_executable (int_hashes_perf int_hashes_perf.cpp) -target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io) - -add_executable (simple_cache simple_cache.cpp) -target_link_libraries (simple_cache PRIVATE common) - -add_executable (compact_array compact_array.cpp) -target_link_libraries (compact_array PRIVATE clickhouse_common_io) - -add_executable (radix_sort radix_sort.cpp) -target_link_libraries (radix_sort PRIVATE clickhouse_common_io) -target_include_directories(radix_sort SYSTEM PRIVATE ${PDQSORT_INCLUDE_DIR}) - -add_executable (arena_with_free_lists arena_with_free_lists.cpp) -target_link_libraries (arena_with_free_lists PRIVATE dbms) - -add_executable (lru_hash_map_perf lru_hash_map_perf.cpp) -target_link_libraries (lru_hash_map_perf PRIVATE dbms) - -add_executable (thread_creation_latency thread_creation_latency.cpp) -target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) - -add_executable (array_cache array_cache.cpp) -target_link_libraries (array_cache PRIVATE clickhouse_common_io) - -add_executable (space_saving space_saving.cpp) -target_link_libraries (space_saving PRIVATE clickhouse_common_io) - -add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) -target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) -target_link_libraries (integer_hash_tables_and_hashes PRIVATE dbms abseil_swiss_tables) - -add_executable (integer_hash_tables_benchmark integer_hash_tables_benchmark.cpp) -target_include_directories (integer_hash_tables_benchmark SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) -target_link_libraries (integer_hash_tables_benchmark PRIVATE dbms abseil_swiss_tables) - -add_executable (cow_columns cow_columns.cpp) -target_link_libraries (cow_columns PRIVATE clickhouse_common_io) - -add_executable (cow_compositions cow_compositions.cpp) -target_link_libraries (cow_compositions PRIVATE clickhouse_common_io) - -add_executable (stopwatch stopwatch.cpp) -target_link_libraries (stopwatch PRIVATE clickhouse_common_io) - -add_executable (symbol_index symbol_index.cpp) -target_link_libraries (symbol_index PRIVATE clickhouse_common_io) - -add_executable (chaos_sanitizer chaos_sanitizer.cpp) -target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io) - -if (OS_LINUX) - add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) - target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) -endif() - -add_executable (procfs_metrics_provider_perf procfs_metrics_provider_perf.cpp) -target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io) - -add_executable (average average.cpp) -target_link_libraries (average PRIVATE clickhouse_common_io) - -add_executable (shell_command_inout shell_command_inout.cpp) -target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) - -if (ENABLE_FUZZING) - add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) - target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp deleted file mode 100644 index 556ed13bb04..00000000000 --- a/src/Common/examples/YAML_fuzzer.cpp +++ /dev/null @@ -1,39 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include - -extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) -{ - /// How to test: - /// build ClickHouse with YAML_fuzzer.cpp - /// ./YAML_fuzzer YAML_CORPUS - /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - char file_name[L_tmpnam]; - if (!std::tmpnam(file_name)) - { - std::cerr << "Cannot create temp file!\n"; - return 1; - } - std::string input = std::string(reinterpret_cast(data), size); - - { - std::ofstream temp_file(file_name); - temp_file << input; - } - - try - { - DB::YAMLParser::parse(std::string(file_name)); - } - catch (...) - { - std::cerr << "YAML_fuzzer failed: " << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << std::endl; - return 1; - } - return 0; -} diff --git a/src/Compression/CMakeLists.txt b/src/Compression/CMakeLists.txt index 390835f17ae..3c97ef1b807 100644 --- a/src/Compression/CMakeLists.txt +++ b/src/Compression/CMakeLists.txt @@ -1,3 +1,14 @@ -if(ENABLE_EXAMPLES) +if (ENABLE_FUZZING) + include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") + add_headers_and_sources(fuzz_compression .) + + # Remove this file, because it has dependencies on DataTypes + list(REMOVE_ITEM ${fuzz_compression_sources} CompressionFactoryAdditions.cpp) + + add_library(fuzz_compression ${fuzz_compression_headers} ${fuzz_compression_sources}) + target_link_libraries(fuzz_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4) +endif() + +if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index bb2b00a56ef..95602086c29 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -27,8 +27,6 @@ namespace ErrorCodes extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } -static constexpr auto DEFAULT_CODEC_NAME = "Default"; - CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const { return default_codec; @@ -49,184 +47,6 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std } } -void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const -{ - if (family_name.empty()) - throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); - - if (level) - { - auto literal = std::make_shared(static_cast(*level)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs); - } - else - { - auto identifier = std::make_shared(Poco::toUpper(family_name)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs); - } -} - -ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const -{ - if (const auto * func = ast->as()) - { - ASTPtr codecs_descriptions = std::make_shared(); - - bool is_compression = false; - bool has_none = false; - std::optional generic_compression_codec_pos; - std::set post_processing_codecs; - - bool can_substitute_codec_arguments = true; - for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) - { - const auto & inner_codec_ast = func->arguments->children[i]; - String codec_family_name; - ASTPtr codec_arguments; - if (const auto * family_name = inner_codec_ast->as()) - { - codec_family_name = family_name->name(); - codec_arguments = {}; - } - else if (const auto * ast_func = inner_codec_ast->as()) - { - codec_family_name = ast_func->name; - codec_arguments = ast_func->arguments; - } - else - throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - /// Default codec replaced with current default codec which may depend on different - /// settings (and properties of data) in runtime. - CompressionCodecPtr result_codec; - if (codec_family_name == DEFAULT_CODEC_NAME) - { - if (codec_arguments != nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); - - result_codec = default_codec; - codecs_descriptions->children.emplace_back(std::make_shared(DEFAULT_CODEC_NAME)); - } - else - { - if (column_type) - { - CompressionCodecPtr prev_codec; - IDataType::StreamCallbackWithType callback = [&]( - const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) - { - if (ISerialization::isSpecialCompressionAllowed(substream_path)) - { - result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); - - /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. - /// We cannot substitute parameters for such codecs. - if (prev_codec && prev_codec->getHash() != result_codec->getHash()) - can_substitute_codec_arguments = false; - prev_codec = result_codec; - } - }; - - ISerialization::SubstreamPath stream_path; - column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path); - - if (!result_codec) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); - } - else - { - result_codec = getImpl(codec_family_name, codec_arguments, nullptr); - } - - if (!allow_experimental_codecs && result_codec->isExperimental()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Codec {} is experimental and not meant to be used in production." - " You can enable it with the 'allow_experimental_codecs' setting.", - codec_family_name); - - codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); - } - - is_compression |= result_codec->isCompression(); - has_none |= result_codec->isNone(); - - if (!generic_compression_codec_pos && result_codec->isGenericCompression()) - generic_compression_codec_pos = i; - - if (result_codec->isPostProcessing()) - post_processing_codecs.insert(i); - } - - String codec_description = queryToString(codecs_descriptions); - - if (sanity_check) - { - if (codecs_descriptions->children.size() > 1 && has_none) - throw Exception( - "It does not make sense to have codec NONE along with other compression codecs: " + codec_description - + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - /// Allow to explicitly specify single NONE codec if user don't want any compression. - /// But applying other transformations solely without compression (e.g. Delta) does not make sense. - /// It's okay to apply post-processing codecs solely without anything else. - if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size()) - throw Exception( - "Compression codec " + codec_description - + " does not compress anything." - " You may want to add generic compression algorithm after other transformations, like: " - + codec_description - + ", LZ4." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - /// It does not make sense to apply any non-post-processing codecs - /// after post-processing one. - if (!post_processing_codecs.empty() && - *post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size()) - throw Exception("The combination of compression codecs " + codec_description + " is meaningless," - " because it does not make sense to apply any non-post-processing codecs after" - " post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'" - " to skip this check).", ErrorCodes::BAD_ARGUMENTS); - - /// It does not make sense to apply any transformations after generic compression algorithm - /// So, generic compression can be only one and only at the end. - if (generic_compression_codec_pos && - *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size()) - throw Exception("The combination of compression codecs " + codec_description + " is meaningless," - " because it does not make sense to apply any transformations after generic compression algorithm." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); - - } - - /// For columns with nested types like Tuple(UInt32, UInt64) we - /// obviously cannot substitute parameters for codecs which depend on - /// data type, because for the first column Delta(4) is suitable and - /// Delta(8) for the second. So we should leave codec description as is - /// and deduce them in get method for each subtype separately. For all - /// other types it's better to substitute parameters, for better - /// readability and backward compatibility. - if (can_substitute_codec_arguments) - { - std::shared_ptr result = std::make_shared(); - result->name = "CODEC"; - result->arguments = codecs_descriptions; - return result; - } - else - { - return ast; - } - } - - throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); -} - CompressionCodecPtr CompressionCodecFactory::get( const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index cdbb663935a..f00e5071990 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -14,6 +14,8 @@ namespace DB { +static constexpr auto DEFAULT_CODEC_NAME = "Default"; + class ICompressionCodec; using CompressionCodecPtr = std::shared_ptr; diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp new file mode 100644 index 00000000000..f569b10bc55 --- /dev/null +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -0,0 +1,212 @@ +/** + * This file contains a part of CompressionCodecFactory methods definitions and + * is needed only because they have dependencies on DataTypes. + * They are not useful for fuzzers, so we leave them in other translation unit. + */ + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int UNKNOWN_CODEC; +} + + +void CompressionCodecFactory::validateCodec( + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const +{ + if (family_name.empty()) + throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); + + if (level) + { + auto literal = std::make_shared(static_cast(*level)); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), + {}, sanity_check, allow_experimental_codecs); + } + else + { + auto identifier = std::make_shared(Poco::toUpper(family_name)); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), + {}, sanity_check, allow_experimental_codecs); + } +} + +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( + const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const +{ + if (const auto * func = ast->as()) + { + ASTPtr codecs_descriptions = std::make_shared(); + + bool is_compression = false; + bool has_none = false; + std::optional generic_compression_codec_pos; + std::set post_processing_codecs; + + bool can_substitute_codec_arguments = true; + for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) + { + const auto & inner_codec_ast = func->arguments->children[i]; + String codec_family_name; + ASTPtr codec_arguments; + if (const auto * family_name = inner_codec_ast->as()) + { + codec_family_name = family_name->name(); + codec_arguments = {}; + } + else if (const auto * ast_func = inner_codec_ast->as()) + { + codec_family_name = ast_func->name; + codec_arguments = ast_func->arguments; + } + else + throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + + /// Default codec replaced with current default codec which may depend on different + /// settings (and properties of data) in runtime. + CompressionCodecPtr result_codec; + if (codec_family_name == DEFAULT_CODEC_NAME) + { + if (codec_arguments != nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); + + result_codec = default_codec; + codecs_descriptions->children.emplace_back(std::make_shared(DEFAULT_CODEC_NAME)); + } + else + { + if (column_type) + { + CompressionCodecPtr prev_codec; + IDataType::StreamCallbackWithType callback = [&]( + const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + { + if (ISerialization::isSpecialCompressionAllowed(substream_path)) + { + result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); + + /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. + /// We cannot substitute parameters for such codecs. + if (prev_codec && prev_codec->getHash() != result_codec->getHash()) + can_substitute_codec_arguments = false; + prev_codec = result_codec; + } + }; + + ISerialization::SubstreamPath stream_path; + column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path); + + if (!result_codec) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); + } + else + { + result_codec = getImpl(codec_family_name, codec_arguments, nullptr); + } + + if (!allow_experimental_codecs && result_codec->isExperimental()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Codec {} is experimental and not meant to be used in production." + " You can enable it with the 'allow_experimental_codecs' setting.", + codec_family_name); + + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); + } + + is_compression |= result_codec->isCompression(); + has_none |= result_codec->isNone(); + + if (!generic_compression_codec_pos && result_codec->isGenericCompression()) + generic_compression_codec_pos = i; + + if (result_codec->isPostProcessing()) + post_processing_codecs.insert(i); + } + + String codec_description = queryToString(codecs_descriptions); + + if (sanity_check) + { + if (codecs_descriptions->children.size() > 1 && has_none) + throw Exception( + "It does not make sense to have codec NONE along with other compression codecs: " + codec_description + + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// Allow to explicitly specify single NONE codec if user don't want any compression. + /// But applying other transformations solely without compression (e.g. Delta) does not make sense. + /// It's okay to apply post-processing codecs solely without anything else. + if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size()) + throw Exception( + "Compression codec " + codec_description + + " does not compress anything." + " You may want to add generic compression algorithm after other transformations, like: " + + codec_description + + ", LZ4." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// It does not make sense to apply any non-post-processing codecs + /// after post-processing one. + if (!post_processing_codecs.empty() && + *post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size()) + throw Exception("The combination of compression codecs " + codec_description + " is meaningless," + " because it does not make sense to apply any non-post-processing codecs after" + " post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'" + " to skip this check).", ErrorCodes::BAD_ARGUMENTS); + + /// It does not make sense to apply any transformations after generic compression algorithm + /// So, generic compression can be only one and only at the end. + if (generic_compression_codec_pos && + *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size()) + throw Exception("The combination of compression codecs " + codec_description + " is meaningless," + " because it does not make sense to apply any transformations after generic compression algorithm." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); + + } + + /// For columns with nested types like Tuple(UInt32, UInt64) we + /// obviously cannot substitute parameters for codecs which depend on + /// data type, because for the first column Delta(4) is suitable and + /// Delta(8) for the second. So we should leave codec description as is + /// and deduce them in get method for each subtype separately. For all + /// other types it's better to substitute parameters, for better + /// readability and backward compatibility. + if (can_substitute_codec_arguments) + { + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = codecs_descriptions; + return result; + } + else + { + return ast; + } + } + + throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); +} + + + +} diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index c0679557ec9..4b727a49861 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -26,23 +26,6 @@ namespace ErrorCodes IDataType::~IDataType() = default; -String IDataType::getName() const -{ - if (custom_name) - { - return custom_name->getName(); - } - else - { - return doGetName(); - } -} - -String IDataType::doGetName() const -{ - return getFamilyName(); -} - void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint) { /// Update the average value size hint if amount of read rows isn't too small diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c4f04282487..13fecb82d68 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -55,7 +55,13 @@ public: /// static constexpr bool is_parametric = false; /// Name of data type (examples: UInt64, Array(String)). - String getName() const; + String getName() const + { + if (custom_name) + return custom_name->getName(); + else + return doGetName(); + } /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; @@ -98,7 +104,7 @@ public: void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback) const { enumerateStreams(serialization, callback, {}); } protected: - virtual String doGetName() const; + virtual String doGetName() const { return getFamilyName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const; diff --git a/src/Storages/CMakeLists.txt b/src/Storages/CMakeLists.txt index ff22e9fa9e1..6d4f18d34b1 100644 --- a/src/Storages/CMakeLists.txt +++ b/src/Storages/CMakeLists.txt @@ -1,6 +1,10 @@ add_subdirectory(MergeTree) add_subdirectory(System) -if(ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index 59d44829363..cb03ae751e3 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -23,10 +23,3 @@ target_link_libraries (transform_part_zk_nodes string_utils ) -if (ENABLE_FUZZING) - add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.cpp) - target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) - - add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) - target_link_libraries (columns_description_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..93d3d2926bd --- /dev/null +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -0,0 +1,11 @@ + +add_executable (mergetree_checksum_fuzzer + mergetree_checksum_fuzzer.cpp + "${ClickHouse_SOURCE_DIR}/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp" + "${ClickHouse_SOURCE_DIR}/src/Compression/CompressedReadBuffer.cpp" + "${ClickHouse_SOURCE_DIR}/src/Compression/CompressedWriteBuffer.cpp" +) +target_link_libraries (mergetree_checksum_fuzzer PRIVATE clickhouse_common_io fuzz_compression ${LIB_FUZZING_ENGINE}) + +add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) +target_link_libraries (columns_description_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Storages/examples/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp similarity index 100% rename from src/Storages/examples/columns_description_fuzzer.cpp rename to src/Storages/fuzzers/columns_description_fuzzer.cpp diff --git a/src/Storages/examples/mergetree_checksum_fuzzer.cpp b/src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp similarity index 100% rename from src/Storages/examples/mergetree_checksum_fuzzer.cpp rename to src/Storages/fuzzers/mergetree_checksum_fuzzer.cpp From 3cd689c1689201e5953f273582e28c354ec773c4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 02:27:27 +0000 Subject: [PATCH 61/64] Moved fuzzers to another directory --- src/Compression/CMakeLists.txt | 4 ++++ src/Compression/examples/CMakeLists.txt | 5 ----- src/Compression/fuzzers/CMakeLists.txt | 2 ++ .../compressed_buffer_fuzzer.cpp | 0 src/Core/CMakeLists.txt | 4 ++++ src/Core/examples/CMakeLists.txt | 5 ----- src/Core/fuzzers/CMakeLists.txt | 2 ++ .../{examples => fuzzers}/names_and_types_fuzzer.cpp | 0 src/Parsers/CMakeLists.txt | 4 ++++ src/Parsers/examples/CMakeLists.txt | 11 ----------- src/Parsers/fuzzers/CMakeLists.txt | 8 ++++++++ .../{examples => fuzzers}/create_parser_fuzzer.cpp | 0 src/Parsers/{examples => fuzzers}/lexer_fuzzer.cpp | 0 .../{examples => fuzzers}/select_parser_fuzzer.cpp | 0 14 files changed, 24 insertions(+), 21 deletions(-) create mode 100644 src/Compression/fuzzers/CMakeLists.txt rename src/Compression/{examples => fuzzers}/compressed_buffer_fuzzer.cpp (100%) create mode 100644 src/Core/fuzzers/CMakeLists.txt rename src/Core/{examples => fuzzers}/names_and_types_fuzzer.cpp (100%) create mode 100644 src/Parsers/fuzzers/CMakeLists.txt rename src/Parsers/{examples => fuzzers}/create_parser_fuzzer.cpp (100%) rename src/Parsers/{examples => fuzzers}/lexer_fuzzer.cpp (100%) rename src/Parsers/{examples => fuzzers}/select_parser_fuzzer.cpp (100%) diff --git a/src/Compression/CMakeLists.txt b/src/Compression/CMakeLists.txt index 3c97ef1b807..34369d8dbc8 100644 --- a/src/Compression/CMakeLists.txt +++ b/src/Compression/CMakeLists.txt @@ -12,3 +12,7 @@ endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/Compression/examples/CMakeLists.txt b/src/Compression/examples/CMakeLists.txt index f47c4404cf6..3cfc0ccb7dc 100644 --- a/src/Compression/examples/CMakeLists.txt +++ b/src/Compression/examples/CMakeLists.txt @@ -3,8 +3,3 @@ target_link_libraries (compressed_buffer PRIVATE dbms) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) target_link_libraries (cached_compressed_read_buffer PRIVATE dbms) - -if (ENABLE_FUZZING) - add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) - target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..74bf2d2649b --- /dev/null +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) +target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression clickhouse_common_io ${LIB_FUZZING_ENGINE}) diff --git a/src/Compression/examples/compressed_buffer_fuzzer.cpp b/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp similarity index 100% rename from src/Compression/examples/compressed_buffer_fuzzer.cpp rename to src/Compression/fuzzers/compressed_buffer_fuzzer.cpp diff --git a/src/Core/CMakeLists.txt b/src/Core/CMakeLists.txt index a6176efc7f3..85e2008753d 100644 --- a/src/Core/CMakeLists.txt +++ b/src/Core/CMakeLists.txt @@ -1,3 +1,7 @@ if (ENABLE_EXAMPLES) add_subdirectory(examples) endif () + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index cd6450633ff..6b07dfbbfa6 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -8,11 +8,6 @@ target_link_libraries (field PRIVATE dbms) add_executable (string_ref_hash string_ref_hash.cpp) target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) -if (ENABLE_FUZZING) - add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) - target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) -endif () - add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) if(USE_SSL) diff --git a/src/Core/fuzzers/CMakeLists.txt b/src/Core/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..a5416035010 --- /dev/null +++ b/src/Core/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) +target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Core/examples/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp similarity index 100% rename from src/Core/examples/names_and_types_fuzzer.cpp rename to src/Core/fuzzers/names_and_types_fuzzer.cpp diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index 5aaa5c32f92..a20dd3567a9 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -12,3 +12,7 @@ endif () if(ENABLE_EXAMPLES) add_subdirectory(examples) endif() + +if (ENABLE_FUZZING) + add_subdirectory(fuzzers) +endif() diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 3ff40b3cc3b..3e1d6ae559f 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -8,14 +8,3 @@ target_link_libraries(select_parser PRIVATE clickhouse_parsers) add_executable(create_parser create_parser.cpp ${SRCS}) target_link_libraries(create_parser PRIVATE clickhouse_parsers) - -if (ENABLE_FUZZING) - add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS}) - target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) - - add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS}) - target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) - - add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) - target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..0dd541e663f --- /dev/null +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -0,0 +1,8 @@ +add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS}) +target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) + +add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS}) +target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) + +add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) +target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) diff --git a/src/Parsers/examples/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp similarity index 100% rename from src/Parsers/examples/create_parser_fuzzer.cpp rename to src/Parsers/fuzzers/create_parser_fuzzer.cpp diff --git a/src/Parsers/examples/lexer_fuzzer.cpp b/src/Parsers/fuzzers/lexer_fuzzer.cpp similarity index 100% rename from src/Parsers/examples/lexer_fuzzer.cpp rename to src/Parsers/fuzzers/lexer_fuzzer.cpp diff --git a/src/Parsers/examples/select_parser_fuzzer.cpp b/src/Parsers/fuzzers/select_parser_fuzzer.cpp similarity index 100% rename from src/Parsers/examples/select_parser_fuzzer.cpp rename to src/Parsers/fuzzers/select_parser_fuzzer.cpp From 9d5cab4d9f360cc1abb344ff5f8a676c2543ae6d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 02:30:28 +0000 Subject: [PATCH 62/64] Better --- src/Common/examples/CMakeLists.txt | 82 ++++++++++++++++++++++++++++++ 1 file changed, 82 insertions(+) create mode 100644 src/Common/examples/CMakeLists.txt diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt new file mode 100644 index 00000000000..64d28fec5c2 --- /dev/null +++ b/src/Common/examples/CMakeLists.txt @@ -0,0 +1,82 @@ +add_executable (hashes_test hashes_test.cpp) +target_link_libraries (hashes_test PRIVATE clickhouse_common_io ${CITYHASH_LIBRARIES}) +if(OPENSSL_CRYPTO_LIBRARY) + target_link_libraries (hashes_test PRIVATE ${OPENSSL_CRYPTO_LIBRARY}) +endif() + +add_executable (sip_hash_perf sip_hash_perf.cpp) +target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) + +add_executable (small_table small_table.cpp) +target_link_libraries (small_table PRIVATE clickhouse_common_io) + +add_executable (parallel_aggregation parallel_aggregation.cpp) +target_link_libraries (parallel_aggregation PRIVATE dbms) + +add_executable (parallel_aggregation2 parallel_aggregation2.cpp) +target_link_libraries (parallel_aggregation2 PRIVATE dbms) + +add_executable (int_hashes_perf int_hashes_perf.cpp) +target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io) + +add_executable (simple_cache simple_cache.cpp) +target_link_libraries (simple_cache PRIVATE common) + +add_executable (compact_array compact_array.cpp) +target_link_libraries (compact_array PRIVATE clickhouse_common_io) + +add_executable (radix_sort radix_sort.cpp) +target_link_libraries (radix_sort PRIVATE clickhouse_common_io) +target_include_directories(radix_sort SYSTEM PRIVATE ${PDQSORT_INCLUDE_DIR}) + +add_executable (arena_with_free_lists arena_with_free_lists.cpp) +target_link_libraries (arena_with_free_lists PRIVATE dbms) + +add_executable (lru_hash_map_perf lru_hash_map_perf.cpp) +target_link_libraries (lru_hash_map_perf PRIVATE dbms) + +add_executable (thread_creation_latency thread_creation_latency.cpp) +target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) + +add_executable (array_cache array_cache.cpp) +target_link_libraries (array_cache PRIVATE clickhouse_common_io) + +add_executable (space_saving space_saving.cpp) +target_link_libraries (space_saving PRIVATE clickhouse_common_io) + +add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) +target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +target_link_libraries (integer_hash_tables_and_hashes PRIVATE dbms abseil_swiss_tables) + +add_executable (integer_hash_tables_benchmark integer_hash_tables_benchmark.cpp) +target_include_directories (integer_hash_tables_benchmark SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +target_link_libraries (integer_hash_tables_benchmark PRIVATE dbms abseil_swiss_tables) + +add_executable (cow_columns cow_columns.cpp) +target_link_libraries (cow_columns PRIVATE clickhouse_common_io) + +add_executable (cow_compositions cow_compositions.cpp) +target_link_libraries (cow_compositions PRIVATE clickhouse_common_io) + +add_executable (stopwatch stopwatch.cpp) +target_link_libraries (stopwatch PRIVATE clickhouse_common_io) + +add_executable (symbol_index symbol_index.cpp) +target_link_libraries (symbol_index PRIVATE clickhouse_common_io) + +add_executable (chaos_sanitizer chaos_sanitizer.cpp) +target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io) + +if (OS_LINUX) + add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) + target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) +endif() + +add_executable (procfs_metrics_provider_perf procfs_metrics_provider_perf.cpp) +target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io) + +add_executable (average average.cpp) +target_link_libraries (average PRIVATE clickhouse_common_io) + +add_executable (shell_command_inout shell_command_inout.cpp) +target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) From 894e56fd99400e58d2f86ab056419d2202233eaf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 21 Aug 2021 10:50:10 +0300 Subject: [PATCH 63/64] Update PULL_REQUEST_TEMPLATE.md --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index d3fac8670e8..a2930beb89f 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -3,7 +3,7 @@ I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla Changelog category (leave one): - New Feature - Improvement -- Bug Fix +- Bug Fix (user-visible misbehaviour in official stable or prestable release) - Performance Improvement - Backward Incompatible Change - Build/Testing/Packaging Improvement From 64821c41d92d62e73c15a02a61eecf81b57d738c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 13:00:15 +0000 Subject: [PATCH 64/64] Style --- src/Compression/CompressionFactory.cpp | 1 - src/Compression/CompressionFactoryAdditions.cpp | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 95602086c29..4dbc72c22f5 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNKNOWN_CODEC; - extern const int BAD_ARGUMENTS; extern const int UNEXPECTED_AST_STRUCTURE; extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index f569b10bc55..bff294d1ea5 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -26,6 +26,8 @@ namespace ErrorCodes { extern const int UNEXPECTED_AST_STRUCTURE; extern const int UNKNOWN_CODEC; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } @@ -49,6 +51,7 @@ void CompressionCodecFactory::validateCodec( } } + ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const { @@ -208,5 +211,4 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( } - }