From d120fdf5953f0db43138ad1b2face732c01dafe3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Sep 2021 21:42:41 +0300 Subject: [PATCH 01/89] check if query context exist before using cache --- src/Interpreters/Context.cpp | 1 + .../InterpreterSelectWithUnionQuery.cpp | 8 ++++++++ .../01162_strange_mutations.reference | 6 ++++++ .../0_stateless/01162_strange_mutations.sh | 19 +++++++++++++++++++ 4 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/01162_strange_mutations.reference create mode 100755 tests/queries/0_stateless/01162_strange_mutations.sh diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7e43343ab34..78f3e8440d2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2478,6 +2478,7 @@ void Context::setFormatSchemaPath(const String & path) Context::SampleBlockCache & Context::getSampleBlockCache() const { + assert(hasQueryContext()); return getQueryContext()->sample_block_cache; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 4aeaa9e4f13..e7ea08e557d 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -222,6 +222,14 @@ InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default; Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, ContextPtr context_, bool is_subquery) { + if (!context_->hasQueryContext()) + { + if (is_subquery) + return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); + else + return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); + } + auto & cache = context_->getSampleBlockCache(); /// Using query string because query_ptr changes for every internal SELECT auto key = queryToString(query_ptr_); diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference new file mode 100644 index 00000000000..64572fe3446 --- /dev/null +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -0,0 +1,6 @@ +1 +2 +0 +1 +2 +0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh new file mode 100755 index 00000000000..6e19f81c3ef --- /dev/null +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_DATABASE', '1')") + +for engine in "${engines[@]}" +do + $CLICKHOUSE_CLIENT -q "drop table if exists t" + $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine order by n" + $CLICKHOUSE_CLIENT -q "insert into t values (1)" + $CLICKHOUSE_CLIENT -q "insert into t values (2)" + $CLICKHOUSE_CLIENT -q "select * from t order by n" + $CLICKHOUSE_CLIENT -q "alter table t delete where n global in (select * from (select * from t))" + $CLICKHOUSE_CLIENT -q "select count() from t" + $CLICKHOUSE_CLIENT -q "drop table t" +done From b702f7cbff638963e1d5afc1c4689f74d062d322 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Sep 2021 13:37:44 +0300 Subject: [PATCH 02/89] preserve table alias when adding default database --- src/Interpreters/AddDefaultDatabaseVisitor.h | 7 ++++++- .../queries/0_stateless/01162_strange_mutations.reference | 2 ++ tests/queries/0_stateless/01162_strange_mutations.sh | 6 ++++-- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index fe3edc00957..858608acdbe 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -105,7 +105,12 @@ private: void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const { if (!identifier.compound()) - ast = std::make_shared(database_name, identifier.name()); + { + auto qualified_identifier = std::make_shared(database_name, identifier.name()); + if (!identifier.alias.empty()) + qualified_identifier->setAlias(identifier.alias); + ast = qualified_identifier; + } } void visit(ASTSubquery & subquery, ASTPtr &) const diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index 64572fe3446..e09dccd4c4a 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -1,6 +1,8 @@ 1 2 +2 0 1 2 +2 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index 6e19f81c3ef..c4166a88e42 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_DATABASE', '1')") +declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1')") for engine in "${engines[@]}" do @@ -13,7 +13,9 @@ do $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT -q "alter table t delete where n global in (select * from (select * from t))" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" + $CLICKHOUSE_CLIENT -q "select * from t order by n" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" done From bb32432943c1e04f442d6df83dcb31faa0c269c8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 5 Oct 2021 00:13:18 +0300 Subject: [PATCH 03/89] more fixes --- .../MergeTree/MutateFromLogEntryTask.cpp | 6 ++- .../MergeTree/MutateFromLogEntryTask.h | 1 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 6 ++- .../MergeTree/MutatePlainMergeTreeTask.h | 1 + src/Storages/StorageMergeTree.cpp | 49 ++----------------- src/Storages/StorageMergeTree.h | 1 - .../01162_strange_mutations.reference | 24 +++++++++ .../0_stateless/01162_strange_mutations.sh | 17 +++++-- 8 files changed, 55 insertions(+), 50 deletions(-) diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 7cca7012a2c..a6c70e1db4f 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -75,9 +75,13 @@ std::pair MutateFromLogEntry stopwatch_ptr = std::make_unique(); + fake_query_context = Context::createCopy(storage.getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); + mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, merge_mutate_entry.get(), - entry.create_time, storage.getContext(), reserved_space, table_lock_holder); + entry.create_time, fake_query_context, reserved_space, table_lock_holder); return {true, [this] (const ExecutionStatus & execution_status) { diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.h b/src/Storages/MergeTree/MutateFromLogEntryTask.h index 2fa4f0c0388..9aaad14dd4c 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.h +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.h @@ -42,6 +42,7 @@ private: MergeTreeData::MutableDataPartPtr new_part{nullptr}; FutureMergedMutatedPartPtr future_mutated_part{nullptr}; + ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; }; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 71128b31cdb..b7768531373 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -43,9 +43,13 @@ void MutatePlainMergeTreeTask::prepare() merge_list_entry.get()); }; + fake_query_context = Context::createCopy(storage.getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); + mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_part, metadata_snapshot, merge_mutate_entry->commands, merge_list_entry.get(), - time(nullptr), storage.getContext(), merge_mutate_entry->tagger->reserved_space, table_lock_holder); + time(nullptr), fake_query_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); } bool MutatePlainMergeTreeTask::executeStep() diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index e8f210f4175..7e332ad3334 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -74,6 +74,7 @@ private: IExecutableTask::TaskResultCallback task_result_callback; + ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index faa16ac875b..3829b3813aa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -927,12 +927,16 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( { try { + auto fake_query_context = Context::createCopy(getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, getContext(), false); + shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false); commands_size += interpreter.evaluateCommandsSize(); } catch (...) { + tryLogCurrentException(log); MergeTreeMutationEntry & entry = it->second; entry.latest_fail_time = time(nullptr); entry.latest_fail_reason = getCurrentExceptionMessage(false); @@ -965,49 +969,6 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( return {}; } -bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) -{ - auto & future_part = merge_mutate_entry.future_part; - - auto merge_list_entry = getContext()->getMergeList().insert(getStorageID(), future_part); - Stopwatch stopwatch; - MutableDataPartPtr new_part; - - auto write_part_log = [&] (const ExecutionStatus & execution_status) - { - writePartLog( - PartLogElement::MUTATE_PART, - execution_status, - stopwatch.elapsed(), - future_part->name, - new_part, - future_part->parts, - merge_list_entry.get()); - }; - - try - { - auto task = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry.commands, merge_list_entry.get(), - time(nullptr), getContext(), merge_mutate_entry.tagger->reserved_space, table_lock_holder); - - new_part = executeHere(task); - - renameTempPartAndReplace(new_part); - - updateMutationEntriesErrors(future_part, true, ""); - write_part_log({}); - } - catch (...) - { - updateMutationEntriesErrors(future_part, false, getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); - throw; - } - - return true; -} - bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) //-V657 { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 08cce514d71..11379359844 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -178,7 +178,6 @@ private: std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); - bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( const DataPartPtr & part, diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index e09dccd4c4a..4c925f849d6 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -1,8 +1,32 @@ +MergeTree 1 2 2 0 +50 6225 0 +0 +50 6225 1900 +ReplicatedMergeTree 1 2 2 0 +50 6225 0 +2 +50 6225 0 +Memory +1 +2 +2 +0 +50 6225 0 +0 +50 6225 1900 +Join +1 +2 +2 +0 +50 6225 0 +0 +50 6225 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index c4166a88e42..c75bee47e4e 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -4,18 +4,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1')") +declare -a engines=("MergeTree order by n" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') order by n" "Memory" "Join(ALL, FULL, n)") + +$CLICKHOUSE_CLIENT -q "CREATE OR REPLACE VIEW t1 AS SELECT number * 10 AS id, number * 100 AS value FROM numbers(20)" for engine in "${engines[@]}" do $CLICKHOUSE_CLIENT -q "drop table if exists t" - $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine order by n" + $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine" + $CLICKHOUSE_CLIENT -q "select engine from system.tables where database=currentDatabase() and name='t'" $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" + + $CLICKHOUSE_CLIENT -q "drop table if exists test" + $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" + $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" + # FIXME it's not clear if the following query should fail or not + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -c "Unknown function" + $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" + $CLICKHOUSE_CLIENT -q "drop table test" done From 07113cb6f63781eda49ab25475c59a5c89093d86 Mon Sep 17 00:00:00 2001 From: Federico Ceratto Date: Thu, 7 Oct 2021 12:50:56 +0100 Subject: [PATCH 04/89] Link FAQ from Debian installation --- docs/en/getting-started/install.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 508cd51e9f8..06186842809 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -27,10 +27,11 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun {% include 'install/deb.sh' %} ``` -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). +You can replace `stable` with `lts` or `testing` to use different [“release trains”](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). + #### Packages {#packages} - `clickhouse-common-static` — Installs ClickHouse compiled binary files. From afd69ef8336356c4e38505c8881dec58e9b6169a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Oct 2021 15:04:54 +0300 Subject: [PATCH 05/89] fix check for nondeterministic mutations --- src/Interpreters/MutationsInterpreter.cpp | 41 ++++++++++++------- .../01162_strange_mutations.reference | 6 +-- .../0_stateless/01162_strange_mutations.sh | 20 +++++++-- 3 files changed, 44 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 540d5c76c97..b1c578854a7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -54,24 +54,33 @@ public: { ContextPtr context; std::optional nondeterministic_function_name; + bool subquery = false; }; - static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & child) + static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/) { - return child != nullptr; + return true; } static void visit(const ASTPtr & node, Data & data) { - if (data.nondeterministic_function_name) + if (data.nondeterministic_function_name || data.subquery) return; - if (const auto * function = typeid_cast(node.get())) + if (node->as()) + { + /// We cannot determine if subquery is deterministic or not, + /// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1 + data.subquery = true; + } + else if (const auto * function = typeid_cast(node.get())) { /// Property of being deterministic for lambda expression is completely determined /// by the contents of its definition, so we just proceed to it. if (function->name != "lambda") { + /// NOTE It may be an aggregate function, so get(...) may throw. + /// However, an aggregate function can be used only in subquery and we do not go into subquery. const auto func = FunctionFactory::instance().get(function->name, data.context); if (!func->isDeterministic()) data.nondeterministic_function_name = func->getName(); @@ -81,10 +90,11 @@ public: }; using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor; +using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data; -std::optional findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context) +FirstNonDeterministicFunctionData findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context) { - FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt}; + FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt, false}; switch (command.type) { @@ -94,7 +104,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast); if (finder_data.nondeterministic_function_name) - return finder_data.nondeterministic_function_name; + return finder_data; /// Currently UPDATE and DELETE both always have predicates so we can use fallthrough [[fallthrough]]; @@ -105,7 +115,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman auto predicate_ast = command.predicate->clone(); FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast); - return finder_data.nondeterministic_function_name; + return finder_data; } default: @@ -918,12 +928,15 @@ void MutationsInterpreter::validate() { for (const auto & command : commands) { - const auto nondeterministic_func_name = findFirstNonDeterministicFunctionName(command, context); - if (nondeterministic_func_name) - throw Exception( - "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions! " - "Function '" + *nondeterministic_func_name + "' is non-deterministic", - ErrorCodes::BAD_ARGUMENTS); + const auto nondeterministic_func_data = findFirstNonDeterministicFunctionName(command, context); + if (nondeterministic_func_data.subquery) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ALTER UPDATE/ALTER DELETE statement with subquery may be nondeterministic, " + "see allow_nondeterministic_mutations setting"); + + if (nondeterministic_func_data.nondeterministic_function_name) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " + "Function '{}' is non-deterministic", *nondeterministic_func_data.nondeterministic_function_name); } } diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index 4c925f849d6..55f17cfe464 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -4,7 +4,6 @@ MergeTree 2 0 50 6225 0 -0 50 6225 1900 ReplicatedMergeTree 1 @@ -12,15 +11,13 @@ ReplicatedMergeTree 2 0 50 6225 0 -2 -50 6225 0 +50 6225 1900 Memory 1 2 2 0 50 6225 0 -0 50 6225 1900 Join 1 @@ -28,5 +25,4 @@ Join 2 0 50 6225 0 -0 50 6225 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index c75bee47e4e..fecb1b8d8c0 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -16,17 +16,29 @@ do $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "alter table t + delete where n global in (select * from (select * from t where n global in (1::Int32)))" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "alter table t + delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" $CLICKHOUSE_CLIENT -q "drop table if exists test" $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" - # FIXME it's not clear if the following query should fail or not - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -c "Unknown function" + if [[ $engine == *"ReplicatedMergeTree"* ]]; then + $CLICKHOUSE_CLIENT -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" + elif [[ $engine == *"Join"* ]]; then + $CLICKHOUSE_CLIENT -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "Table engine Join supports only DELETE mutations" + else + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" + fi $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" $CLICKHOUSE_CLIENT -q "drop table test" done From 3a3ea9f19cf1fa3fab45b01a2630f42f0bef145b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 7 Oct 2021 18:42:25 +0300 Subject: [PATCH 06/89] Update 00652_replicated_mutations_default_database_zookeeper.sh --- .../00652_replicated_mutations_default_database_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh index 3f5b8d570a6..0ac5a2f748a 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --multiquery << EOF +${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 --multiquery << EOF DROP TABLE IF EXISTS mutations_r1; DROP TABLE IF EXISTS for_subquery; From f953cb85f850da07bb5316af5007d51bf4bee1fe Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Fri, 8 Oct 2021 21:01:14 +0300 Subject: [PATCH 07/89] Check --- .gitmodules | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..d8c5aa640a8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -212,7 +212,8 @@ url = https://github.com/ClickHouse-Extras/libpq [submodule "contrib/boringssl"] path = contrib/boringssl - url = https://github.com/ClickHouse-Extras/boringssl.git + url = https://github.com/FArthur-cmd/boringssl.git + branch = update_BoringSSL [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git From 351f2a3a842c6ddb63c09958c5e2397c1f42a093 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 11 Oct 2021 14:53:26 +0300 Subject: [PATCH 08/89] Update BorinSSL --- .gitmodules | 3 +- contrib/boringssl-cmake/CMakeLists.txt | 49 ++++++++++++++++++-------- 2 files changed, 35 insertions(+), 17 deletions(-) diff --git a/.gitmodules b/.gitmodules index d8c5aa640a8..74d1049ce01 100644 --- a/.gitmodules +++ b/.gitmodules @@ -212,8 +212,7 @@ url = https://github.com/ClickHouse-Extras/libpq [submodule "contrib/boringssl"] path = contrib/boringssl - url = https://github.com/FArthur-cmd/boringssl.git - branch = update_BoringSSL + url = https://github.com/ClickHouse-Extras/boringssl.git [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git diff --git a/contrib/boringssl-cmake/CMakeLists.txt b/contrib/boringssl-cmake/CMakeLists.txt index 4502d6e9d42..474e32f3b91 100644 --- a/contrib/boringssl-cmake/CMakeLists.txt +++ b/contrib/boringssl-cmake/CMakeLists.txt @@ -4,7 +4,7 @@ # This file is created by generate_build_files.py and edited accordingly. -cmake_minimum_required(VERSION 3.0) +cmake_minimum_required(VERSION 3.5) project(BoringSSL LANGUAGES C CXX) @@ -20,12 +20,7 @@ if(CMAKE_COMPILER_IS_GNUCXX OR CLANG) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") endif() - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-common") - if((CMAKE_C_COMPILER_VERSION VERSION_GREATER "4.8.99") OR CLANG) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -std=c11") - else() - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -std=c99") - endif() + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-common -std=c11") endif() # pthread_rwlock_t requires a feature flag. @@ -55,7 +50,7 @@ add_definitions(-DBORINGSSL_IMPLEMENTATION) # builds. if(NOT OPENSSL_NO_ASM AND CMAKE_OSX_ARCHITECTURES) list(LENGTH CMAKE_OSX_ARCHITECTURES NUM_ARCHES) - if(NOT ${NUM_ARCHES} EQUAL 1) + if(NOT NUM_ARCHES EQUAL 1) message(FATAL_ERROR "Universal binaries not supported.") endif() list(GET CMAKE_OSX_ARCHITECTURES 0 CMAKE_SYSTEM_PROCESSOR) @@ -78,7 +73,13 @@ elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "AMD64") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "x86") set(ARCH "x86") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "i386") - set(ARCH "x86") + # cmake uses `uname -p` to set the system processor, but Solaris + # systems support multiple architectures. + if((${CMAKE_SYSTEM_NAME} STREQUAL "SunOS") AND CMAKE_SIZEOF_VOID_P EQUAL 8) + set(ARCH "x86_64") + else() + set(ARCH "x86") + endif() elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "i686") set(ARCH "x86") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "aarch64") @@ -289,6 +290,21 @@ set( mac-x86_64/crypto/test/trampoline-x86_64.S ) +set( + CRYPTO_win_aarch64_SOURCES + + win-aarch64/crypto/chacha/chacha-armv8.S + win-aarch64/crypto/fipsmodule/aesv8-armx64.S + win-aarch64/crypto/fipsmodule/armv8-mont.S + win-aarch64/crypto/fipsmodule/ghash-neon-armv8.S + win-aarch64/crypto/fipsmodule/ghashv8-armx64.S + win-aarch64/crypto/fipsmodule/sha1-armv8.S + win-aarch64/crypto/fipsmodule/sha256-armv8.S + win-aarch64/crypto/fipsmodule/sha512-armv8.S + win-aarch64/crypto/fipsmodule/vpaes-armv8.S + win-aarch64/crypto/test/trampoline-armv8.S +) + set( CRYPTO_win_x86_SOURCES @@ -331,9 +347,9 @@ set( win-x86_64/crypto/test/trampoline-x86_64.asm ) -if(APPLE AND ${ARCH} STREQUAL "aarch64") +if(APPLE AND ARCH STREQUAL "aarch64") set(CRYPTO_ARCH_SOURCES ${CRYPTO_ios_aarch64_SOURCES}) -elseif(APPLE AND ${ARCH} STREQUAL "arm") +elseif(APPLE AND ARCH STREQUAL "arm") set(CRYPTO_ARCH_SOURCES ${CRYPTO_ios_arm_SOURCES}) elseif(APPLE) set(CRYPTO_ARCH_SOURCES ${CRYPTO_mac_${ARCH}_SOURCES}) @@ -360,6 +376,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_object.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_octet.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_print.c" + "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_strex.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_strnid.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_time.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_type.c" @@ -389,6 +406,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/bio/printf.c" "${BORINGSSL_SOURCE_DIR}/crypto/bio/socket.c" "${BORINGSSL_SOURCE_DIR}/crypto/bio/socket_helper.c" + "${BORINGSSL_SOURCE_DIR}/crypto/blake2/blake2.c" "${BORINGSSL_SOURCE_DIR}/crypto/bn_extra/bn_asn1.c" "${BORINGSSL_SOURCE_DIR}/crypto/bn_extra/convert.c" "${BORINGSSL_SOURCE_DIR}/crypto/buf/buf.c" @@ -413,6 +431,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/conf/conf.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-fuchsia.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-linux.c" + "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-win.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-arm-linux.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-arm.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-intel.c" @@ -452,7 +471,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/ex_data.c" "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/bcm.c" "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/fips_shared_support.c" - "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/is_fips.c" "${BORINGSSL_SOURCE_DIR}/crypto/hkdf/hkdf.c" "${BORINGSSL_SOURCE_DIR}/crypto/hpke/hpke.c" "${BORINGSSL_SOURCE_DIR}/crypto/hrss/hrss.c" @@ -499,13 +517,13 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/trust_token/voprf.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_digest.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_sign.c" - "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_strex.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_verify.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/algorithm.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/asn1_gen.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/by_dir.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/by_file.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/i2d_pr.c" + "${BORINGSSL_SOURCE_DIR}/crypto/x509/name_print.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/rsa_pss.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/t_crl.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/t_req.c" @@ -519,7 +537,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_ext.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_lu.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_obj.c" - "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_r2x.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_req.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_set.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_trs.c" @@ -589,6 +606,8 @@ add_library( "${BORINGSSL_SOURCE_DIR}/ssl/d1_srtp.cc" "${BORINGSSL_SOURCE_DIR}/ssl/dtls_method.cc" "${BORINGSSL_SOURCE_DIR}/ssl/dtls_record.cc" + "${BORINGSSL_SOURCE_DIR}/ssl/encrypted_client_hello.cc" + "${BORINGSSL_SOURCE_DIR}/ssl/extensions.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handoff.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handshake.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handshake_client.cc" @@ -611,7 +630,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/ssl/ssl_versions.cc" "${BORINGSSL_SOURCE_DIR}/ssl/ssl_x509.cc" "${BORINGSSL_SOURCE_DIR}/ssl/t1_enc.cc" - "${BORINGSSL_SOURCE_DIR}/ssl/t1_lib.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_both.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_client.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_enc.cc" @@ -633,6 +651,7 @@ add_executable( "${BORINGSSL_SOURCE_DIR}/tool/digest.cc" "${BORINGSSL_SOURCE_DIR}/tool/fd.cc" "${BORINGSSL_SOURCE_DIR}/tool/file.cc" + "${BORINGSSL_SOURCE_DIR}/tool/generate_ech.cc" "${BORINGSSL_SOURCE_DIR}/tool/generate_ed25519.cc" "${BORINGSSL_SOURCE_DIR}/tool/genrsa.cc" "${BORINGSSL_SOURCE_DIR}/tool/pkcs12.cc" From 0d1fcdf9fcf1e6717c671f78c529fc331048a86b Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 11 Oct 2021 15:08:40 +0300 Subject: [PATCH 09/89] add submodule update --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index a6a2e2ab3e4..4c787e9d70c 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit a6a2e2ab3e44d97ce98e51c558e989f211de7eb3 +Subproject commit 4c787e9d70c370d51baea714e7b73910be2a4c28 From 7b9358a59721749b7a60d917d1793969760a9df5 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Tue, 12 Oct 2021 03:21:31 +0300 Subject: [PATCH 10/89] adding codegen fuzzer + code generation script. new contrib added: libprotobuf-mutator --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find/libprotobuf-mutator.cmake | 11 + contrib/CMakeLists.txt | 4 + contrib/libprotobuf-mutator | 1 + .../libprotobuf-mutator-cmake/CMakeLists.txt | 17 + src/Parsers/fuzzers/CMakeLists.txt | 5 + .../fuzzers/codegen_fuzzer/CMakeLists.txt | 13 + .../codegen_fuzzer/clickhouse-template.g | 121 + .../fuzzers/codegen_fuzzer/clickhouse.g | 524 ++++ .../codegen_fuzzer/codegen_select_fuzzer.cpp | 40 + src/Parsers/fuzzers/codegen_fuzzer/gen.py | 249 ++ src/Parsers/fuzzers/codegen_fuzzer/out.cpp | 2189 +++++++++++++++++ src/Parsers/fuzzers/codegen_fuzzer/out.proto | 519 ++++ src/Parsers/fuzzers/codegen_fuzzer/update.sh | 30 + 15 files changed, 3727 insertions(+) create mode 100644 cmake/find/libprotobuf-mutator.cmake create mode 160000 contrib/libprotobuf-mutator create mode 100644 contrib/libprotobuf-mutator-cmake/CMakeLists.txt create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/gen.py create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.cpp create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.proto create mode 100755 src/Parsers/fuzzers/codegen_fuzzer/update.sh diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..5d226cddd29 100644 --- a/.gitmodules +++ b/.gitmodules @@ -249,3 +249,6 @@ [submodule "contrib/magic_enum"] path = contrib/magic_enum url = https://github.com/Neargye/magic_enum +[submodule "contrib/libprotobuf-mutator"] + path = contrib/libprotobuf-mutator + url = https://github.com/google/libprotobuf-mutator diff --git a/CMakeLists.txt b/CMakeLists.txt index 3f553c5c26d..843beec01c6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -562,6 +562,7 @@ include (cmake/find/cassandra.cmake) include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) +include (cmake/find/libprotobuf-mutator.cmake) set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/libprotobuf-mutator.cmake b/cmake/find/libprotobuf-mutator.cmake new file mode 100644 index 00000000000..8aa595230cd --- /dev/null +++ b/cmake/find/libprotobuf-mutator.cmake @@ -0,0 +1,11 @@ +option(USE_LIBPROTOBUF_MUTATOR "Enable libprotobuf-mutator" ${ENABLE_FUZZING}) + +if (NOT USE_LIBPROTOBUF_MUTATOR) + return() +endif() + +set(LibProtobufMutator_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator") + +if (NOT EXISTS "${LibProtobufMutator_SOURCE_DIR}/README.md") + message (ERROR "submodule contrib/libprotobuf-mutator is missing. to fix try run: \n git submodule update --init --recursive") +endif() diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 140cc0846ec..98231856aee 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -49,6 +49,10 @@ add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (nanodbc-cmake) +if (ENABLE_FUZZING) + add_subdirectory (libprotobuf-mutator-cmake) +endif() + if (USE_YAML_CPP) add_subdirectory (yaml-cpp-cmake) endif() diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator new file mode 160000 index 00000000000..ffd86a32874 --- /dev/null +++ b/contrib/libprotobuf-mutator @@ -0,0 +1 @@ +Subproject commit ffd86a32874e5c08a143019aad1aaf0907294c9f diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt new file mode 100644 index 00000000000..93eafc85b7d --- /dev/null +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -0,0 +1,17 @@ +set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator) +set(NO_FUZZING_FLAGS "-fno-sanitize=fuzzer -fsanitize-coverage=0") + +add_library(protobuf-mutator + ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_macro.cc + ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_mutator.cc + ${LIBRARY_DIR}/src/binary_format.cc + ${LIBRARY_DIR}/src/mutator.cc + ${LIBRARY_DIR}/src/text_format.cc + ${LIBRARY_DIR}/src/utf8_fix.cc) + +target_include_directories(protobuf-mutator BEFORE PRIVATE "${LIBRARY_DIR}") +# target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") + +target_link_libraries(protobuf-mutator ${PROTOBUF_LIBRARIES}) +set_target_properties(protobuf-mutator PROPERTIES + COMPILE_FLAGS "${NO_FUZZING_FLAGS}") diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt index 0dd541e663f..2840dc72c0a 100644 --- a/src/Parsers/fuzzers/CMakeLists.txt +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -6,3 +6,8 @@ target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZ add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) + +string(REPLACE " -Werror" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") +add_subdirectory(codegen_fuzzer) +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror") + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt new file mode 100644 index 00000000000..f55bb3b3fb9 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -0,0 +1,13 @@ +find_package(Protobuf REQUIRED) + +protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS "out.proto") +set(FUZZER_SRCS codegen_select_fuzzer.cpp out.cpp ${PROTO_SRCS} ${PROTO_HDRS}) + +set(CMAKE_INCLUDE_CURRENT_DIR TRUE) + +add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) + +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") +target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) \ No newline at end of file diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g new file mode 100644 index 00000000000..79fd775b1da --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g @@ -0,0 +1,121 @@ +" "; +" "; +" "; +";"; + + +"(" $1 ")"; +"(" $1 ", " $2 ")"; +"(" $1 ", " $2 ", " $3 ")"; + +$1 ", " $2 ; +$1 ", " $2 ", " $3 ; +$1 ", " $2 ", " $3 ", " $4 ; +$1 ", " $2 ", " $3 ", " $4 ", " $5 ; + +"[" $1 ", " $2 "]"; +"[" $1 ", " $2 ", " $3 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; + +$0 "(" $1 ")"; +$0 "(" $1 ", " $2 ")"; +$0 "(" $1 ", " $2 ", " $3 ")"; + +$1 " as " $2 ; + + +// TODO: add more clickhouse specific stuff +"SELECT " $1 " FROM " $2 " WHERE " $3 ; +"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; +"SELECT " $1 " FROM " $2 " SORT BY " $3 ; +"SELECT " $1 " FROM " $2 " LIMIT " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 ; +"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; +"SELECT " $1 " INTO OUTFILE " $2 ; + +"WITH " $1 " AS " $2 ; + +"{" $1 ":" $2 "}"; +"[" $1 "," $2 "]"; +"[]"; + + +" x "; +"x"; +" `x` "; +"`x`"; + +" \"value\" "; +"\"value\""; +" 0 "; +"0"; +"1"; +"2"; +"123123123123123123"; +"182374019873401982734091873420923123123123123123"; +"1e-1"; +"1.1"; +"\"\""; +" '../../../../../../../../../etc/passwd' "; + +"/"; +"="; +"=="; +"!="; +"<>"; +"<"; +"<="; +">"; +">="; +"<<"; +"|<<"; +"&"; +"|"; +"||"; +"<|"; +"|>"; +"+"; +"-"; +"~"; +"*"; +"/"; +"\\"; +"%"; +""; +"."; +","; +","; +","; +","; +","; +","; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"?"; +":"; +"@"; +"@@"; +"$"; +"\""; +"`"; +"{"; +"}"; +"^"; +"::"; +"->"; +"]"; +"["; + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g new file mode 100644 index 00000000000..edd5acf513d --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g @@ -0,0 +1,524 @@ +" "; +" "; +" "; +";"; + + +"(" $1 ")"; +"(" $1 ", " $2 ")"; +"(" $1 ", " $2 ", " $3 ")"; + +$1 ", " $2 ; +$1 ", " $2 ", " $3 ; +$1 ", " $2 ", " $3 ", " $4 ; +$1 ", " $2 ", " $3 ", " $4 ", " $5 ; + +"[" $1 ", " $2 "]"; +"[" $1 ", " $2 ", " $3 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; + +$0 "(" $1 ")"; +$0 "(" $1 ", " $2 ")"; +$0 "(" $1 ", " $2 ", " $3 ")"; + +$1 " as " $2 ; + + +// TODO: add more clickhouse specific stuff +"SELECT " $1 " FROM " $2 " WHERE " $3 ; +"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; +"SELECT " $1 " FROM " $2 " SORT BY " $3 ; +"SELECT " $1 " FROM " $2 " LIMIT " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 ; +"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; +"SELECT " $1 " INTO OUTFILE " $2 ; + +"WITH " $1 " AS " $2 ; + +"{" $1 ":" $2 "}"; +"[" $1 "," $2 "]"; +"[]"; + + +" x "; +"x"; +" `x` "; +"`x`"; + +" \"value\" "; +"\"value\""; +" 0 "; +"0"; +"1"; +"2"; +"123123123123123123"; +"182374019873401982734091873420923123123123123123"; +"1e-1"; +"1.1"; +"\"\""; +" '../../../../../../../../../etc/passwd' "; + +"/"; +"="; +"=="; +"!="; +"<>"; +"<"; +"<="; +">"; +">="; +"<<"; +"|<<"; +"&"; +"|"; +"||"; +"<|"; +"|>"; +"+"; +"-"; +"~"; +"*"; +"/"; +"\\"; +"%"; +""; +"."; +","; +","; +","; +","; +","; +","; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"?"; +":"; +"@"; +"@@"; +"$"; +"\""; +"`"; +"{"; +"}"; +"^"; +"::"; +"->"; +"]"; +"["; + +" ADD "; +" ADD COLUMN "; +" ADD CONSTRAINT "; +" ADD INDEX "; +" AFTER "; +" AggregateFunction "; +" aggThrow "; +" ALL "; +" ALTER LIVE VIEW "; +" ALTER TABLE "; +" and "; +" ANTI "; +" any "; +" anyHeavy "; +" anyLast "; +" argMax "; +" argMin "; +" array "; +" Array "; +" arrayAll "; +" arrayAUC "; +" arrayCompact "; +" arrayConcat "; +" arrayCount "; +" arrayCumSum "; +" arrayCumSumNonNegative "; +" arrayDifference "; +" arrayDistinct "; +" arrayElement "; +" arrayEnumerate "; +" arrayEnumerateDense "; +" arrayEnumerateDenseRanked "; +" arrayEnumerateUniq "; +" arrayEnumerateUniqRanked "; +" arrayExists "; +" arrayFill "; +" arrayFilter "; +" arrayFirst "; +" arrayFirstIndex "; +" arrayFlatten "; +" arrayIntersect "; +" arrayJoin "; +" ARRAY JOIN "; +" arrayMap "; +" arrayPopBack "; +" arrayPopFront "; +" arrayPushBack "; +" arrayPushFront "; +" arrayReduce "; +" arrayReduceInRanges "; +" arrayResize "; +" arrayReverse "; +" arrayReverseFill "; +" arrayReverseSort "; +" arrayReverseSplit "; +" arraySlice "; +" arraySort "; +" arraySplit "; +" arraySum "; +" arrayUniq "; +" arrayWithConstant "; +" arrayZip "; +" AS "; +" ASC "; +" ASCENDING "; +" ASOF "; +" AST "; +" ATTACH "; +" ATTACH PART "; +" ATTACH PARTITION "; +" avg "; +" avgWeighted "; +" basename "; +" BETWEEN "; +" BOTH "; +" boundingRatio "; +" BY "; +" CAST "; +" categoricalInformationValue "; +" CHECK "; +" CHECK TABLE "; +" CLEAR COLUMN "; +" CLEAR INDEX "; +" COLLATE "; +" COLUMNS "; +" COMMENT COLUMN "; +" CONSTRAINT "; +" corr "; +" corrStable "; +" count "; +" countEqual "; +" covarPop "; +" covarPopStable "; +" covarSamp "; +" covarSampStable "; +" CREATE "; +" CROSS "; +" CUBE "; +" cutFragment "; +" cutQueryString "; +" cutQueryStringAndFragment "; +" cutToFirstSignificantSubdomain "; +" cutURLParameter "; +" cutWWW "; +" D "; +" DATABASE "; +" DATABASES "; +" Date "; +" DATE "; +" DATE_ADD "; +" DATEADD "; +" DATE_DIFF "; +" DATEDIFF "; +" DATE_SUB "; +" DATESUB "; +" DateTime "; +" DateTime64 "; +" DAY "; +" DD "; +" Decimal "; +" Decimal128 "; +" Decimal32 "; +" Decimal64 "; +" decodeURLComponent "; +" DEDUPLICATE "; +" DELETE "; +" DELETE WHERE "; +" DESC "; +" DESCENDING "; +" DESCRIBE "; +" DETACH "; +" DETACH PARTITION "; +" DICTIONARIES "; +" DICTIONARY "; +" DISTINCT "; +" domain "; +" domainWithoutWWW "; +" DROP "; +" DROP COLUMN "; +" DROP CONSTRAINT "; +" DROP DETACHED PART "; +" DROP DETACHED PARTITION "; +" DROP INDEX "; +" DROP PARTITION "; +" emptyArrayToSingle "; +" ENGINE "; +" entropy "; +" Enum "; +" Enum16 "; +" Enum8 "; +" EVENTS "; +" EXCHANGE TABLES "; +" EXISTS "; +" EXTRACT "; +" extractURLParameter "; +" extractURLParameterNames "; +" extractURLParameters "; +" FETCH PARTITION "; +" FETCH PART "; +" FINAL "; +" FIRST "; +" firstSignificantSubdomain "; +" FixedString "; +" Float32 "; +" Float64 "; +" FOR "; +" ForEach "; +" FORMAT "; +" fragment "; +" FREEZE "; +" FROM "; +" FULL "; +" FUNCTION "; +" __getScalar "; +" GLOBAL "; +" GRANULARITY "; +" groupArray "; +" groupArrayInsertAt "; +" groupArrayMovingAvg "; +" groupArrayMovingSum "; +" groupArraySample "; +" groupBitAnd "; +" groupBitmap "; +" groupBitmapAnd "; +" groupBitmapOr "; +" groupBitmapXor "; +" groupBitOr "; +" groupBitXor "; +" GROUP BY "; +" groupUniqArray "; +" has "; +" hasAll "; +" hasAny "; +" HAVING "; +" HH "; +" histogram "; +" HOUR "; +" ID "; +" if "; +" IF EXISTS "; +" IF NOT EXISTS "; +" IN "; +" INDEX "; +" indexOf "; +" INNER "; +" IN PARTITION "; +" INSERT INTO "; +" Int16 "; +" Int32 "; +" Int64 "; +" Int8 "; +" INTERVAL "; +" IntervalDay "; +" IntervalHour "; +" IntervalMinute "; +" IntervalMonth "; +" IntervalQuarter "; +" IntervalSecond "; +" IntervalWeek "; +" IntervalYear "; +" INTO OUTFILE "; +" JOIN "; +" kurtPop "; +" kurtSamp "; +" LAST "; +" LAYOUT "; +" LEADING "; +" LEFT "; +" LEFT ARRAY JOIN "; +" length "; +" LIFETIME "; +" LIKE "; +" LIMIT "; +" LIVE "; +" LOCAL "; +" LowCardinality "; +" LTRIM "; +" M "; +" MATERIALIZED "; +" MATERIALIZE INDEX "; +" MATERIALIZE TTL "; +" max "; +" maxIntersections "; +" maxIntersectionsPosition "; +" Merge "; +" MI "; +" min "; +" MINUTE "; +" MM "; +" MODIFY "; +" MODIFY COLUMN "; +" MODIFY ORDER BY "; +" MODIFY QUERY "; +" MODIFY SETTING "; +" MODIFY TTL "; +" MONTH "; +" MOVE PART "; +" MOVE PARTITION "; +" movingXXX "; +" N "; +" NAME "; +" Nested "; +" NO DELAY "; +" NONE "; +" not "; +" nothing "; +" Nothing "; +" Null "; +" Nullable "; +" NULLS "; +" OFFSET "; +" ON "; +" ONLY "; +" OPTIMIZE TABLE "; +" ORDER BY "; +" OR REPLACE "; +" OUTER "; +" PARTITION "; +" PARTITION BY "; +" path "; +" pathFull "; +" POPULATE "; +" PREWHERE "; +" PRIMARY KEY "; +" protocol "; +" Q "; +" QQ "; +" QUARTER "; +" queryString "; +" queryStringAndFragment "; +" range "; +" REFRESH "; +" RENAME COLUMN "; +" RENAME TABLE "; +" REPLACE PARTITION "; +" Resample "; +" RESUME "; +" retention "; +" RIGHT "; +" ROLLUP "; +" RTRIM "; +" S "; +" SAMPLE "; +" SAMPLE BY "; +" SECOND "; +" SELECT "; +" SEMI "; +" sequenceCount "; +" sequenceMatch "; +" SET "; +" SETTINGS "; +" SHOW "; +" SHOW PROCESSLIST "; +" simpleLinearRegression "; +" skewPop "; +" skewSamp "; +" SOURCE "; +" SQL_TSI_DAY "; +" SQL_TSI_HOUR "; +" SQL_TSI_MINUTE "; +" SQL_TSI_MONTH "; +" SQL_TSI_QUARTER "; +" SQL_TSI_SECOND "; +" SQL_TSI_WEEK "; +" SQL_TSI_YEAR "; +" SS "; +" State "; +" stddevPop "; +" stddevPopStable "; +" stddevSamp "; +" stddevSampStable "; +" STEP "; +" stochasticLinearRegression "; +" stochasticLogisticRegression "; +" String "; +" SUBSTRING "; +" sum "; +" sumKahan "; +" sumMap "; +" sumMapFiltered "; +" sumMapFilteredWithOverflow "; +" sumMapWithOverflow "; +" sumWithOverflow "; +" SUSPEND "; +" TABLE "; +" TABLES "; +" TEMPORARY "; +" TIMESTAMP "; +" TIMESTAMP_ADD "; +" TIMESTAMPADD "; +" TIMESTAMP_DIFF "; +" TIMESTAMPDIFF "; +" TIMESTAMP_SUB "; +" TIMESTAMPSUB "; +" TO "; +" TO DISK "; +" TOP "; +" topK "; +" topKWeighted "; +" topLevelDomain "; +" TO TABLE "; +" TOTALS "; +" TO VOLUME "; +" TRAILING "; +" TRIM "; +" TRUNCATE "; +" TTL "; +" Tuple "; +" TYPE "; +" UInt16 "; +" UInt32 "; +" UInt64 "; +" UInt8 "; +" uniq "; +" uniqCombined "; +" uniqCombined64 "; +" uniqExact "; +" uniqHLL12 "; +" uniqUpTo "; +" UPDATE "; +" URLHierarchy "; +" URLPathHierarchy "; +" USE "; +" USING "; +" UUID "; +" VALUES "; +" varPop "; +" varPopStable "; +" varSamp "; +" varSampStable "; +" VIEW "; +" WATCH "; +" WEEK "; +" WHERE "; +" windowFunnel "; +" WITH "; +" WITH FILL "; +" WITH TIES "; +" WK "; +" WW "; +" YEAR "; +" YY "; +" YYYY "; diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp new file mode 100644 index 00000000000..a68fac4f585 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -0,0 +1,40 @@ + +#include +#include + +#include +#include +#include +#include + +#include + +#include "out.pb.h" + + +void GenerateSentence(const Sentence&, std::string &, int); + + +DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { + static std::string input; + input.reserve(4096); + + GenerateSentence(main, input, 0); + if (input.size()) { + + std::cout << input << std::endl; + + DB::ParserQueryWithOutput parser(input.data() + input.size()); + try { + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + + DB::WriteBufferFromOStream out(std::cerr, 4096); + DB::formatAST(*ast, out); + std::cerr << std::endl; + } catch (...) { + + } + + input.clear(); + } +} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py new file mode 100644 index 00000000000..e96bc6ae9f6 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -0,0 +1,249 @@ +#!/usr/bin/env python3 + +import sys +import string + + +TOKEN_TEXT = 1 +TOKEN_VAR = 2 + +TOKEN_COLON = ':' +TOKEN_SEMI = ';' +TOKEN_OR = '|' +TOKEN_QUESTIONMARK = '?' +TOKEN_ROUND_BRACKET_OPEN = '(' +TOKEN_ROUND_BRACKET_CLOSE = ')' +TOKEN_ASTERISK = '*' +TOKEN_SLASH = '/' + + + + +class TextValue: + def __init__(self, t): + self.t = t + self.slug = None + + def get_slug(self): + if self.slug is not None: + return self.slug + slug = '' + for c in self.t: + slug += c if c in string.ascii_letters else '_' + self.slug = slug + return slug + + def get_name(self): + return f"TextValue_{self.get_slug()}" + + def __repr__(self): + return f"TextValue(\"{self.t}\")" + + +class Var: + def __init__(self, id_): + self.id_ = id_ + + def __repr__(self): + return f"Var({self.id_})" + + +class Parser: + def __init__(self): + self.chains = [] + self.text = None + self.col = 0 + self.line = 1 + self.t = None + self.var_id = -1 + self.cur_tok = None + self.includes = [] + + self.proto = '' + self.cpp = '' + + def parse_file(self, filename): + with open(filename) as f: + self.text = f.read() + + while self.parse_statement() is not None: + pass + + def add_include(self, filename): + self.includes.append(filename) + + def get_next_token(self): + self.skip_ws() + + if not len(self.text): + return None + + if self.text[0] == '"': + return self.parse_txt_value() + + if self.text[0] == '$': + return self.parse_var_value() + + c, self.text = self.text[0], self.text[1:] + self.cur_tok = c + return c + + def parse_var_value(self): + i = self.text.find(' ') + + id_, self.text = self.text[1:i], self.text[i+1:] + self.var_id = int(id_) + self.cur_tok = TOKEN_VAR + return TOKEN_VAR + + def parse_txt_value(self): + if self.text[0] != '"': + raise Exception("parse_txt_value: expected quote at the start") + + self.t = '' + self.text = self.text[1:] + + while self.text[0] != '"': + if self.text[0] == '\\': + if self.text[1] == 'x': + self.t += self.text[:4] + self.text = self.text[4:] + elif self.text[1] in 'nt\\"': + self.t += self.text[:2] + self.text = self.text[2:] + else: + raise Exception(f"parse_txt_value: unknown symbol {self.text[0]}") + else: + c, self.text = self.text[0], self.text[1:] + self.t += c + + self.text = self.text[1:] + self.cur_tok = TOKEN_TEXT + return TOKEN_TEXT + + def skip_ws(self): + while self.text and self.text[0] in string.whitespace: + if self.text[0] == '\n': + self.line += 1 + self.col = 0 + self.text = self.text[1:] + self.col += 1 + if not self.text: + return None + return True + + def skip_line(self): + self.line += 1 + index = self.text.find('\n') + self.text = self.text[index:] + + + def parse_statement(self): + if self.skip_ws() is None: + return None + + self.get_next_token() + if self.cur_tok == TOKEN_SLASH: + self.skip_line() + return TOKEN_SLASH + + chain = [] + while self.cur_tok != TOKEN_SEMI: + if self.cur_tok == TOKEN_TEXT: + chain.append(TextValue(self.t)) + elif self.cur_tok == TOKEN_VAR: + chain.append(Var(self.var_id)) + else: + self.fatal_parsing_error(f"unexpected token {self.tok}") + self.get_next_token() + + if not chain: + self.fatal_parsing_error("empty chains are not allowed") + self.chains.append(chain) + return True + + def generate(self): + self.proto = 'syntax = "proto3";\n\n' + self.cpp = '#include \n#include \n#include \n\n#include \n\n' + + for incl_file in self.includes: + self.cpp += f'#include "{incl_file}"\n' + self.cpp += '\n' + + self.proto += 'message Word {\n' + self.proto += '\tenum Value {\n' + + self.cpp += 'void GenerateWord(const Word&, std::string&, int);\n\n' + + self.cpp += 'void GenerateSentence(const Sentence& stc, std::string &s, int depth) {\n' + self.cpp += '\tfor (int i = 0; i < stc.words_size(); i++ ) {\n' + self.cpp += '\t\tGenerateWord(stc.words(i), s, ++depth);\n' + self.cpp += '\t}\n' + self.cpp += '}\n' + + self.cpp += 'void GenerateWord(const Word& word, std::string &s, int depth) {\n' + + self.cpp += '\tif (depth > 5) return;\n\n' + self.cpp += '\tswitch (word.value()) {\n' + + for idx, chain in enumerate(self.chains): + self.proto += f'\t\tvalue_{idx} = {idx};\n' + + self.cpp += f'\t\tcase {idx}: {{\n' + num_var = 0 + for item in chain: + if isinstance(item, TextValue): + self.cpp += f'\t\t\ts += "{item.t}";\n' + elif isinstance(item, Var): + self.cpp += f'\t\t\tif (word.inner().words_size() > {num_var})\t\t\t\tGenerateWord(word.inner().words({num_var}), s, ++depth);\n' + num_var += 1 + else: + raise Exception("unknown token met during generation") + self.cpp += '\t\t\tbreak;\n\t\t}\n' + self.cpp += '\t\tdefault: break;\n' + + self.cpp += '\t}\n' + + self.proto += '\t}\n' + self.proto += '\tValue value = 1;\n' + self.proto += '\tSentence inner = 2;\n' + self.proto += '}\nmessage Sentence {\n\trepeated Word words = 1;\n}' + + self.cpp += '}\n' + return self.cpp, self.proto + + def fatal_parsing_error(self, msg): + print(f"Line: {self.line}, Col: {self.col}") + raise Exception(f"fatal error during parsing. {msg}") + + +def main(args): + input_file, outfile_cpp, outfile_proto = args + + if not outfile_proto.endswith('.proto'): + raise Exception("outfile_proto (argv[3]) should end with `.proto`") + + include_filename = outfile_proto[:-6] + ".pb.h" + + p = Parser() + p.add_include(include_filename) + p.parse_file(input_file) + + cpp, proto = p.generate() + + proto = proto.replace('\t', ' ' * 4) + cpp = cpp.replace('\t', ' ' * 4) + + with open(outfile_cpp, 'w') as f: + f.write(cpp) + + with open(outfile_proto, 'w') as f: + f.write(proto) + + +if __name__ == '__main__': + if len(sys.argv) < 3: + print(f"Usage {sys.argv[0]} ") + sys.exit(1) + main(sys.argv[1:]) + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp new file mode 100644 index 00000000000..effe6e7821b --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp @@ -0,0 +1,2189 @@ +#include +#include +#include + +#include + +#include "out.pb.h" + +void GenerateWord(const Word&, std::string&, int); + +void GenerateSentence(const Sentence& stc, std::string &s, int depth) { + for (int i = 0; i < stc.words_size(); i++ ) { + GenerateWord(stc.words(i), s, ++depth); + } +} +void GenerateWord(const Word& word, std::string &s, int depth) { + if (depth > 5) return; + + switch (word.value()) { + case 0: { + s += " "; + break; + } + case 1: { + s += " "; + break; + } + case 2: { + s += " "; + break; + } + case 3: { + s += ";"; + break; + } + case 4: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ")"; + break; + } + case 5: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ")"; + break; + } + case 6: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ")"; + break; + } + case 7: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 8: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 9: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 10: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ", "; + if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); + break; + } + case 11: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "]"; + break; + } + case 12: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += "]"; + break; + } + case 13: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += "]"; + break; + } + case 14: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ", "; + if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); + s += "]"; + break; + } + case 15: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ")"; + break; + } + case 16: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ")"; + break; + } + case 17: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ")"; + break; + } + case 18: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " as "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 19: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " WHERE "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 20: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " GROUP BY "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 21: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " SORT BY "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 22: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " LIMIT "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 23: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 24: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " ARRAY JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 25: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += " ON "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 26: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += " USING "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 27: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " INTO OUTFILE "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 28: { + s += "WITH "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " AS "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 29: { + s += "{"; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ":"; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "}"; + break; + } + case 30: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ","; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "]"; + break; + } + case 31: { + s += "[]"; + break; + } + case 32: { + s += " x "; + break; + } + case 33: { + s += "x"; + break; + } + case 34: { + s += " `x` "; + break; + } + case 35: { + s += "`x`"; + break; + } + case 36: { + s += " \"value\" "; + break; + } + case 37: { + s += "\"value\""; + break; + } + case 38: { + s += " 0 "; + break; + } + case 39: { + s += "0"; + break; + } + case 40: { + s += "1"; + break; + } + case 41: { + s += "2"; + break; + } + case 42: { + s += "123123123123123123"; + break; + } + case 43: { + s += "182374019873401982734091873420923123123123123123"; + break; + } + case 44: { + s += "1e-1"; + break; + } + case 45: { + s += "1.1"; + break; + } + case 46: { + s += "\"\""; + break; + } + case 47: { + s += " '../../../../../../../../../etc/passwd' "; + break; + } + case 48: { + s += "/"; + break; + } + case 49: { + s += "="; + break; + } + case 50: { + s += "=="; + break; + } + case 51: { + s += "!="; + break; + } + case 52: { + s += "<>"; + break; + } + case 53: { + s += "<"; + break; + } + case 54: { + s += "<="; + break; + } + case 55: { + s += ">"; + break; + } + case 56: { + s += ">="; + break; + } + case 57: { + s += "<<"; + break; + } + case 58: { + s += "|<<"; + break; + } + case 59: { + s += "&"; + break; + } + case 60: { + s += "|"; + break; + } + case 61: { + s += "||"; + break; + } + case 62: { + s += "<|"; + break; + } + case 63: { + s += "|>"; + break; + } + case 64: { + s += "+"; + break; + } + case 65: { + s += "-"; + break; + } + case 66: { + s += "~"; + break; + } + case 67: { + s += "*"; + break; + } + case 68: { + s += "/"; + break; + } + case 69: { + s += "\\"; + break; + } + case 70: { + s += "%"; + break; + } + case 71: { + s += ""; + break; + } + case 72: { + s += "."; + break; + } + case 73: { + s += ","; + break; + } + case 74: { + s += ","; + break; + } + case 75: { + s += ","; + break; + } + case 76: { + s += ","; + break; + } + case 77: { + s += ","; + break; + } + case 78: { + s += ","; + break; + } + case 79: { + s += "("; + break; + } + case 80: { + s += ")"; + break; + } + case 81: { + s += "("; + break; + } + case 82: { + s += ")"; + break; + } + case 83: { + s += "("; + break; + } + case 84: { + s += ")"; + break; + } + case 85: { + s += "("; + break; + } + case 86: { + s += ")"; + break; + } + case 87: { + s += "("; + break; + } + case 88: { + s += ")"; + break; + } + case 89: { + s += "("; + break; + } + case 90: { + s += ")"; + break; + } + case 91: { + s += "?"; + break; + } + case 92: { + s += ":"; + break; + } + case 93: { + s += "@"; + break; + } + case 94: { + s += "@@"; + break; + } + case 95: { + s += "$"; + break; + } + case 96: { + s += "\""; + break; + } + case 97: { + s += "`"; + break; + } + case 98: { + s += "{"; + break; + } + case 99: { + s += "}"; + break; + } + case 100: { + s += "^"; + break; + } + case 101: { + s += "::"; + break; + } + case 102: { + s += "->"; + break; + } + case 103: { + s += "]"; + break; + } + case 104: { + s += "["; + break; + } + case 105: { + s += " ADD "; + break; + } + case 106: { + s += " ADD COLUMN "; + break; + } + case 107: { + s += " ADD CONSTRAINT "; + break; + } + case 108: { + s += " ADD INDEX "; + break; + } + case 109: { + s += " AFTER "; + break; + } + case 110: { + s += " AggregateFunction "; + break; + } + case 111: { + s += " aggThrow "; + break; + } + case 112: { + s += " ALL "; + break; + } + case 113: { + s += " ALTER LIVE VIEW "; + break; + } + case 114: { + s += " ALTER TABLE "; + break; + } + case 115: { + s += " and "; + break; + } + case 116: { + s += " ANTI "; + break; + } + case 117: { + s += " any "; + break; + } + case 118: { + s += " anyHeavy "; + break; + } + case 119: { + s += " anyLast "; + break; + } + case 120: { + s += " argMax "; + break; + } + case 121: { + s += " argMin "; + break; + } + case 122: { + s += " array "; + break; + } + case 123: { + s += " Array "; + break; + } + case 124: { + s += " arrayAll "; + break; + } + case 125: { + s += " arrayAUC "; + break; + } + case 126: { + s += " arrayCompact "; + break; + } + case 127: { + s += " arrayConcat "; + break; + } + case 128: { + s += " arrayCount "; + break; + } + case 129: { + s += " arrayCumSum "; + break; + } + case 130: { + s += " arrayCumSumNonNegative "; + break; + } + case 131: { + s += " arrayDifference "; + break; + } + case 132: { + s += " arrayDistinct "; + break; + } + case 133: { + s += " arrayElement "; + break; + } + case 134: { + s += " arrayEnumerate "; + break; + } + case 135: { + s += " arrayEnumerateDense "; + break; + } + case 136: { + s += " arrayEnumerateDenseRanked "; + break; + } + case 137: { + s += " arrayEnumerateUniq "; + break; + } + case 138: { + s += " arrayEnumerateUniqRanked "; + break; + } + case 139: { + s += " arrayExists "; + break; + } + case 140: { + s += " arrayFill "; + break; + } + case 141: { + s += " arrayFilter "; + break; + } + case 142: { + s += " arrayFirst "; + break; + } + case 143: { + s += " arrayFirstIndex "; + break; + } + case 144: { + s += " arrayFlatten "; + break; + } + case 145: { + s += " arrayIntersect "; + break; + } + case 146: { + s += " arrayJoin "; + break; + } + case 147: { + s += " ARRAY JOIN "; + break; + } + case 148: { + s += " arrayMap "; + break; + } + case 149: { + s += " arrayPopBack "; + break; + } + case 150: { + s += " arrayPopFront "; + break; + } + case 151: { + s += " arrayPushBack "; + break; + } + case 152: { + s += " arrayPushFront "; + break; + } + case 153: { + s += " arrayReduce "; + break; + } + case 154: { + s += " arrayReduceInRanges "; + break; + } + case 155: { + s += " arrayResize "; + break; + } + case 156: { + s += " arrayReverse "; + break; + } + case 157: { + s += " arrayReverseFill "; + break; + } + case 158: { + s += " arrayReverseSort "; + break; + } + case 159: { + s += " arrayReverseSplit "; + break; + } + case 160: { + s += " arraySlice "; + break; + } + case 161: { + s += " arraySort "; + break; + } + case 162: { + s += " arraySplit "; + break; + } + case 163: { + s += " arraySum "; + break; + } + case 164: { + s += " arrayUniq "; + break; + } + case 165: { + s += " arrayWithConstant "; + break; + } + case 166: { + s += " arrayZip "; + break; + } + case 167: { + s += " AS "; + break; + } + case 168: { + s += " ASC "; + break; + } + case 169: { + s += " ASCENDING "; + break; + } + case 170: { + s += " ASOF "; + break; + } + case 171: { + s += " AST "; + break; + } + case 172: { + s += " ATTACH "; + break; + } + case 173: { + s += " ATTACH PART "; + break; + } + case 174: { + s += " ATTACH PARTITION "; + break; + } + case 175: { + s += " avg "; + break; + } + case 176: { + s += " avgWeighted "; + break; + } + case 177: { + s += " basename "; + break; + } + case 178: { + s += " BETWEEN "; + break; + } + case 179: { + s += " BOTH "; + break; + } + case 180: { + s += " boundingRatio "; + break; + } + case 181: { + s += " BY "; + break; + } + case 182: { + s += " CAST "; + break; + } + case 183: { + s += " categoricalInformationValue "; + break; + } + case 184: { + s += " CHECK "; + break; + } + case 185: { + s += " CHECK TABLE "; + break; + } + case 186: { + s += " CLEAR COLUMN "; + break; + } + case 187: { + s += " CLEAR INDEX "; + break; + } + case 188: { + s += " COLLATE "; + break; + } + case 189: { + s += " COLUMNS "; + break; + } + case 190: { + s += " COMMENT COLUMN "; + break; + } + case 191: { + s += " CONSTRAINT "; + break; + } + case 192: { + s += " corr "; + break; + } + case 193: { + s += " corrStable "; + break; + } + case 194: { + s += " count "; + break; + } + case 195: { + s += " countEqual "; + break; + } + case 196: { + s += " covarPop "; + break; + } + case 197: { + s += " covarPopStable "; + break; + } + case 198: { + s += " covarSamp "; + break; + } + case 199: { + s += " covarSampStable "; + break; + } + case 200: { + s += " CREATE "; + break; + } + case 201: { + s += " CROSS "; + break; + } + case 202: { + s += " CUBE "; + break; + } + case 203: { + s += " cutFragment "; + break; + } + case 204: { + s += " cutQueryString "; + break; + } + case 205: { + s += " cutQueryStringAndFragment "; + break; + } + case 206: { + s += " cutToFirstSignificantSubdomain "; + break; + } + case 207: { + s += " cutURLParameter "; + break; + } + case 208: { + s += " cutWWW "; + break; + } + case 209: { + s += " D "; + break; + } + case 210: { + s += " DATABASE "; + break; + } + case 211: { + s += " DATABASES "; + break; + } + case 212: { + s += " Date "; + break; + } + case 213: { + s += " DATE "; + break; + } + case 214: { + s += " DATE_ADD "; + break; + } + case 215: { + s += " DATEADD "; + break; + } + case 216: { + s += " DATE_DIFF "; + break; + } + case 217: { + s += " DATEDIFF "; + break; + } + case 218: { + s += " DATE_SUB "; + break; + } + case 219: { + s += " DATESUB "; + break; + } + case 220: { + s += " DateTime "; + break; + } + case 221: { + s += " DateTime64 "; + break; + } + case 222: { + s += " DAY "; + break; + } + case 223: { + s += " DD "; + break; + } + case 224: { + s += " Decimal "; + break; + } + case 225: { + s += " Decimal128 "; + break; + } + case 226: { + s += " Decimal32 "; + break; + } + case 227: { + s += " Decimal64 "; + break; + } + case 228: { + s += " decodeURLComponent "; + break; + } + case 229: { + s += " DEDUPLICATE "; + break; + } + case 230: { + s += " DELETE "; + break; + } + case 231: { + s += " DELETE WHERE "; + break; + } + case 232: { + s += " DESC "; + break; + } + case 233: { + s += " DESCENDING "; + break; + } + case 234: { + s += " DESCRIBE "; + break; + } + case 235: { + s += " DETACH "; + break; + } + case 236: { + s += " DETACH PARTITION "; + break; + } + case 237: { + s += " DICTIONARIES "; + break; + } + case 238: { + s += " DICTIONARY "; + break; + } + case 239: { + s += " DISTINCT "; + break; + } + case 240: { + s += " domain "; + break; + } + case 241: { + s += " domainWithoutWWW "; + break; + } + case 242: { + s += " DROP "; + break; + } + case 243: { + s += " DROP COLUMN "; + break; + } + case 244: { + s += " DROP CONSTRAINT "; + break; + } + case 245: { + s += " DROP DETACHED PART "; + break; + } + case 246: { + s += " DROP DETACHED PARTITION "; + break; + } + case 247: { + s += " DROP INDEX "; + break; + } + case 248: { + s += " DROP PARTITION "; + break; + } + case 249: { + s += " emptyArrayToSingle "; + break; + } + case 250: { + s += " ENGINE "; + break; + } + case 251: { + s += " entropy "; + break; + } + case 252: { + s += " Enum "; + break; + } + case 253: { + s += " Enum16 "; + break; + } + case 254: { + s += " Enum8 "; + break; + } + case 255: { + s += " EVENTS "; + break; + } + case 256: { + s += " EXCHANGE TABLES "; + break; + } + case 257: { + s += " EXISTS "; + break; + } + case 258: { + s += " EXTRACT "; + break; + } + case 259: { + s += " extractURLParameter "; + break; + } + case 260: { + s += " extractURLParameterNames "; + break; + } + case 261: { + s += " extractURLParameters "; + break; + } + case 262: { + s += " FETCH PARTITION "; + break; + } + case 263: { + s += " FETCH PART "; + break; + } + case 264: { + s += " FINAL "; + break; + } + case 265: { + s += " FIRST "; + break; + } + case 266: { + s += " firstSignificantSubdomain "; + break; + } + case 267: { + s += " FixedString "; + break; + } + case 268: { + s += " Float32 "; + break; + } + case 269: { + s += " Float64 "; + break; + } + case 270: { + s += " FOR "; + break; + } + case 271: { + s += " ForEach "; + break; + } + case 272: { + s += " FORMAT "; + break; + } + case 273: { + s += " fragment "; + break; + } + case 274: { + s += " FREEZE "; + break; + } + case 275: { + s += " FROM "; + break; + } + case 276: { + s += " FULL "; + break; + } + case 277: { + s += " FUNCTION "; + break; + } + case 278: { + s += " __getScalar "; + break; + } + case 279: { + s += " GLOBAL "; + break; + } + case 280: { + s += " GRANULARITY "; + break; + } + case 281: { + s += " groupArray "; + break; + } + case 282: { + s += " groupArrayInsertAt "; + break; + } + case 283: { + s += " groupArrayMovingAvg "; + break; + } + case 284: { + s += " groupArrayMovingSum "; + break; + } + case 285: { + s += " groupArraySample "; + break; + } + case 286: { + s += " groupBitAnd "; + break; + } + case 287: { + s += " groupBitmap "; + break; + } + case 288: { + s += " groupBitmapAnd "; + break; + } + case 289: { + s += " groupBitmapOr "; + break; + } + case 290: { + s += " groupBitmapXor "; + break; + } + case 291: { + s += " groupBitOr "; + break; + } + case 292: { + s += " groupBitXor "; + break; + } + case 293: { + s += " GROUP BY "; + break; + } + case 294: { + s += " groupUniqArray "; + break; + } + case 295: { + s += " has "; + break; + } + case 296: { + s += " hasAll "; + break; + } + case 297: { + s += " hasAny "; + break; + } + case 298: { + s += " HAVING "; + break; + } + case 299: { + s += " HH "; + break; + } + case 300: { + s += " histogram "; + break; + } + case 301: { + s += " HOUR "; + break; + } + case 302: { + s += " ID "; + break; + } + case 303: { + s += " if "; + break; + } + case 304: { + s += " IF EXISTS "; + break; + } + case 305: { + s += " IF NOT EXISTS "; + break; + } + case 306: { + s += " IN "; + break; + } + case 307: { + s += " INDEX "; + break; + } + case 308: { + s += " indexOf "; + break; + } + case 309: { + s += " INNER "; + break; + } + case 310: { + s += " IN PARTITION "; + break; + } + case 311: { + s += " INSERT INTO "; + break; + } + case 312: { + s += " Int16 "; + break; + } + case 313: { + s += " Int32 "; + break; + } + case 314: { + s += " Int64 "; + break; + } + case 315: { + s += " Int8 "; + break; + } + case 316: { + s += " INTERVAL "; + break; + } + case 317: { + s += " IntervalDay "; + break; + } + case 318: { + s += " IntervalHour "; + break; + } + case 319: { + s += " IntervalMinute "; + break; + } + case 320: { + s += " IntervalMonth "; + break; + } + case 321: { + s += " IntervalQuarter "; + break; + } + case 322: { + s += " IntervalSecond "; + break; + } + case 323: { + s += " IntervalWeek "; + break; + } + case 324: { + s += " IntervalYear "; + break; + } + case 325: { + s += " INTO OUTFILE "; + break; + } + case 326: { + s += " JOIN "; + break; + } + case 327: { + s += " kurtPop "; + break; + } + case 328: { + s += " kurtSamp "; + break; + } + case 329: { + s += " LAST "; + break; + } + case 330: { + s += " LAYOUT "; + break; + } + case 331: { + s += " LEADING "; + break; + } + case 332: { + s += " LEFT "; + break; + } + case 333: { + s += " LEFT ARRAY JOIN "; + break; + } + case 334: { + s += " length "; + break; + } + case 335: { + s += " LIFETIME "; + break; + } + case 336: { + s += " LIKE "; + break; + } + case 337: { + s += " LIMIT "; + break; + } + case 338: { + s += " LIVE "; + break; + } + case 339: { + s += " LOCAL "; + break; + } + case 340: { + s += " LowCardinality "; + break; + } + case 341: { + s += " LTRIM "; + break; + } + case 342: { + s += " M "; + break; + } + case 343: { + s += " MATERIALIZED "; + break; + } + case 344: { + s += " MATERIALIZE INDEX "; + break; + } + case 345: { + s += " MATERIALIZE TTL "; + break; + } + case 346: { + s += " max "; + break; + } + case 347: { + s += " maxIntersections "; + break; + } + case 348: { + s += " maxIntersectionsPosition "; + break; + } + case 349: { + s += " Merge "; + break; + } + case 350: { + s += " MI "; + break; + } + case 351: { + s += " min "; + break; + } + case 352: { + s += " MINUTE "; + break; + } + case 353: { + s += " MM "; + break; + } + case 354: { + s += " MODIFY "; + break; + } + case 355: { + s += " MODIFY COLUMN "; + break; + } + case 356: { + s += " MODIFY ORDER BY "; + break; + } + case 357: { + s += " MODIFY QUERY "; + break; + } + case 358: { + s += " MODIFY SETTING "; + break; + } + case 359: { + s += " MODIFY TTL "; + break; + } + case 360: { + s += " MONTH "; + break; + } + case 361: { + s += " MOVE PART "; + break; + } + case 362: { + s += " MOVE PARTITION "; + break; + } + case 363: { + s += " movingXXX "; + break; + } + case 364: { + s += " N "; + break; + } + case 365: { + s += " NAME "; + break; + } + case 366: { + s += " Nested "; + break; + } + case 367: { + s += " NO DELAY "; + break; + } + case 368: { + s += " NONE "; + break; + } + case 369: { + s += " not "; + break; + } + case 370: { + s += " nothing "; + break; + } + case 371: { + s += " Nothing "; + break; + } + case 372: { + s += " Null "; + break; + } + case 373: { + s += " Nullable "; + break; + } + case 374: { + s += " NULLS "; + break; + } + case 375: { + s += " OFFSET "; + break; + } + case 376: { + s += " ON "; + break; + } + case 377: { + s += " ONLY "; + break; + } + case 378: { + s += " OPTIMIZE TABLE "; + break; + } + case 379: { + s += " ORDER BY "; + break; + } + case 380: { + s += " OR REPLACE "; + break; + } + case 381: { + s += " OUTER "; + break; + } + case 382: { + s += " PARTITION "; + break; + } + case 383: { + s += " PARTITION BY "; + break; + } + case 384: { + s += " path "; + break; + } + case 385: { + s += " pathFull "; + break; + } + case 386: { + s += " POPULATE "; + break; + } + case 387: { + s += " PREWHERE "; + break; + } + case 388: { + s += " PRIMARY KEY "; + break; + } + case 389: { + s += " protocol "; + break; + } + case 390: { + s += " Q "; + break; + } + case 391: { + s += " QQ "; + break; + } + case 392: { + s += " QUARTER "; + break; + } + case 393: { + s += " queryString "; + break; + } + case 394: { + s += " queryStringAndFragment "; + break; + } + case 395: { + s += " range "; + break; + } + case 396: { + s += " REFRESH "; + break; + } + case 397: { + s += " RENAME COLUMN "; + break; + } + case 398: { + s += " RENAME TABLE "; + break; + } + case 399: { + s += " REPLACE PARTITION "; + break; + } + case 400: { + s += " Resample "; + break; + } + case 401: { + s += " RESUME "; + break; + } + case 402: { + s += " retention "; + break; + } + case 403: { + s += " RIGHT "; + break; + } + case 404: { + s += " ROLLUP "; + break; + } + case 405: { + s += " RTRIM "; + break; + } + case 406: { + s += " S "; + break; + } + case 407: { + s += " SAMPLE "; + break; + } + case 408: { + s += " SAMPLE BY "; + break; + } + case 409: { + s += " SECOND "; + break; + } + case 410: { + s += " SELECT "; + break; + } + case 411: { + s += " SEMI "; + break; + } + case 412: { + s += " sequenceCount "; + break; + } + case 413: { + s += " sequenceMatch "; + break; + } + case 414: { + s += " SET "; + break; + } + case 415: { + s += " SETTINGS "; + break; + } + case 416: { + s += " SHOW "; + break; + } + case 417: { + s += " SHOW PROCESSLIST "; + break; + } + case 418: { + s += " simpleLinearRegression "; + break; + } + case 419: { + s += " skewPop "; + break; + } + case 420: { + s += " skewSamp "; + break; + } + case 421: { + s += " SOURCE "; + break; + } + case 422: { + s += " SQL_TSI_DAY "; + break; + } + case 423: { + s += " SQL_TSI_HOUR "; + break; + } + case 424: { + s += " SQL_TSI_MINUTE "; + break; + } + case 425: { + s += " SQL_TSI_MONTH "; + break; + } + case 426: { + s += " SQL_TSI_QUARTER "; + break; + } + case 427: { + s += " SQL_TSI_SECOND "; + break; + } + case 428: { + s += " SQL_TSI_WEEK "; + break; + } + case 429: { + s += " SQL_TSI_YEAR "; + break; + } + case 430: { + s += " SS "; + break; + } + case 431: { + s += " State "; + break; + } + case 432: { + s += " stddevPop "; + break; + } + case 433: { + s += " stddevPopStable "; + break; + } + case 434: { + s += " stddevSamp "; + break; + } + case 435: { + s += " stddevSampStable "; + break; + } + case 436: { + s += " STEP "; + break; + } + case 437: { + s += " stochasticLinearRegression "; + break; + } + case 438: { + s += " stochasticLogisticRegression "; + break; + } + case 439: { + s += " String "; + break; + } + case 440: { + s += " SUBSTRING "; + break; + } + case 441: { + s += " sum "; + break; + } + case 442: { + s += " sumKahan "; + break; + } + case 443: { + s += " sumMap "; + break; + } + case 444: { + s += " sumMapFiltered "; + break; + } + case 445: { + s += " sumMapFilteredWithOverflow "; + break; + } + case 446: { + s += " sumMapWithOverflow "; + break; + } + case 447: { + s += " sumWithOverflow "; + break; + } + case 448: { + s += " SUSPEND "; + break; + } + case 449: { + s += " TABLE "; + break; + } + case 450: { + s += " TABLES "; + break; + } + case 451: { + s += " TEMPORARY "; + break; + } + case 452: { + s += " TIMESTAMP "; + break; + } + case 453: { + s += " TIMESTAMP_ADD "; + break; + } + case 454: { + s += " TIMESTAMPADD "; + break; + } + case 455: { + s += " TIMESTAMP_DIFF "; + break; + } + case 456: { + s += " TIMESTAMPDIFF "; + break; + } + case 457: { + s += " TIMESTAMP_SUB "; + break; + } + case 458: { + s += " TIMESTAMPSUB "; + break; + } + case 459: { + s += " TO "; + break; + } + case 460: { + s += " TO DISK "; + break; + } + case 461: { + s += " TOP "; + break; + } + case 462: { + s += " topK "; + break; + } + case 463: { + s += " topKWeighted "; + break; + } + case 464: { + s += " topLevelDomain "; + break; + } + case 465: { + s += " TO TABLE "; + break; + } + case 466: { + s += " TOTALS "; + break; + } + case 467: { + s += " TO VOLUME "; + break; + } + case 468: { + s += " TRAILING "; + break; + } + case 469: { + s += " TRIM "; + break; + } + case 470: { + s += " TRUNCATE "; + break; + } + case 471: { + s += " TTL "; + break; + } + case 472: { + s += " Tuple "; + break; + } + case 473: { + s += " TYPE "; + break; + } + case 474: { + s += " UInt16 "; + break; + } + case 475: { + s += " UInt32 "; + break; + } + case 476: { + s += " UInt64 "; + break; + } + case 477: { + s += " UInt8 "; + break; + } + case 478: { + s += " uniq "; + break; + } + case 479: { + s += " uniqCombined "; + break; + } + case 480: { + s += " uniqCombined64 "; + break; + } + case 481: { + s += " uniqExact "; + break; + } + case 482: { + s += " uniqHLL12 "; + break; + } + case 483: { + s += " uniqUpTo "; + break; + } + case 484: { + s += " UPDATE "; + break; + } + case 485: { + s += " URLHierarchy "; + break; + } + case 486: { + s += " URLPathHierarchy "; + break; + } + case 487: { + s += " USE "; + break; + } + case 488: { + s += " USING "; + break; + } + case 489: { + s += " UUID "; + break; + } + case 490: { + s += " VALUES "; + break; + } + case 491: { + s += " varPop "; + break; + } + case 492: { + s += " varPopStable "; + break; + } + case 493: { + s += " varSamp "; + break; + } + case 494: { + s += " varSampStable "; + break; + } + case 495: { + s += " VIEW "; + break; + } + case 496: { + s += " WATCH "; + break; + } + case 497: { + s += " WEEK "; + break; + } + case 498: { + s += " WHERE "; + break; + } + case 499: { + s += " windowFunnel "; + break; + } + case 500: { + s += " WITH "; + break; + } + case 501: { + s += " WITH FILL "; + break; + } + case 502: { + s += " WITH TIES "; + break; + } + case 503: { + s += " WK "; + break; + } + case 504: { + s += " WW "; + break; + } + case 505: { + s += " YEAR "; + break; + } + case 506: { + s += " YY "; + break; + } + case 507: { + s += " YYYY "; + break; + } + default: break; + } +} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.proto b/src/Parsers/fuzzers/codegen_fuzzer/out.proto new file mode 100644 index 00000000000..6c8cefce9a3 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/out.proto @@ -0,0 +1,519 @@ +syntax = "proto3"; + +message Word { + enum Value { + value_0 = 0; + value_1 = 1; + value_2 = 2; + value_3 = 3; + value_4 = 4; + value_5 = 5; + value_6 = 6; + value_7 = 7; + value_8 = 8; + value_9 = 9; + value_10 = 10; + value_11 = 11; + value_12 = 12; + value_13 = 13; + value_14 = 14; + value_15 = 15; + value_16 = 16; + value_17 = 17; + value_18 = 18; + value_19 = 19; + value_20 = 20; + value_21 = 21; + value_22 = 22; + value_23 = 23; + value_24 = 24; + value_25 = 25; + value_26 = 26; + value_27 = 27; + value_28 = 28; + value_29 = 29; + value_30 = 30; + value_31 = 31; + value_32 = 32; + value_33 = 33; + value_34 = 34; + value_35 = 35; + value_36 = 36; + value_37 = 37; + value_38 = 38; + value_39 = 39; + value_40 = 40; + value_41 = 41; + value_42 = 42; + value_43 = 43; + value_44 = 44; + value_45 = 45; + value_46 = 46; + value_47 = 47; + value_48 = 48; + value_49 = 49; + value_50 = 50; + value_51 = 51; + value_52 = 52; + value_53 = 53; + value_54 = 54; + value_55 = 55; + value_56 = 56; + value_57 = 57; + value_58 = 58; + value_59 = 59; + value_60 = 60; + value_61 = 61; + value_62 = 62; + value_63 = 63; + value_64 = 64; + value_65 = 65; + value_66 = 66; + value_67 = 67; + value_68 = 68; + value_69 = 69; + value_70 = 70; + value_71 = 71; + value_72 = 72; + value_73 = 73; + value_74 = 74; + value_75 = 75; + value_76 = 76; + value_77 = 77; + value_78 = 78; + value_79 = 79; + value_80 = 80; + value_81 = 81; + value_82 = 82; + value_83 = 83; + value_84 = 84; + value_85 = 85; + value_86 = 86; + value_87 = 87; + value_88 = 88; + value_89 = 89; + value_90 = 90; + value_91 = 91; + value_92 = 92; + value_93 = 93; + value_94 = 94; + value_95 = 95; + value_96 = 96; + value_97 = 97; + value_98 = 98; + value_99 = 99; + value_100 = 100; + value_101 = 101; + value_102 = 102; + value_103 = 103; + value_104 = 104; + value_105 = 105; + value_106 = 106; + value_107 = 107; + value_108 = 108; + value_109 = 109; + value_110 = 110; + value_111 = 111; + value_112 = 112; + value_113 = 113; + value_114 = 114; + value_115 = 115; + value_116 = 116; + value_117 = 117; + value_118 = 118; + value_119 = 119; + value_120 = 120; + value_121 = 121; + value_122 = 122; + value_123 = 123; + value_124 = 124; + value_125 = 125; + value_126 = 126; + value_127 = 127; + value_128 = 128; + value_129 = 129; + value_130 = 130; + value_131 = 131; + value_132 = 132; + value_133 = 133; + value_134 = 134; + value_135 = 135; + value_136 = 136; + value_137 = 137; + value_138 = 138; + value_139 = 139; + value_140 = 140; + value_141 = 141; + value_142 = 142; + value_143 = 143; + value_144 = 144; + value_145 = 145; + value_146 = 146; + value_147 = 147; + value_148 = 148; + value_149 = 149; + value_150 = 150; + value_151 = 151; + value_152 = 152; + value_153 = 153; + value_154 = 154; + value_155 = 155; + value_156 = 156; + value_157 = 157; + value_158 = 158; + value_159 = 159; + value_160 = 160; + value_161 = 161; + value_162 = 162; + value_163 = 163; + value_164 = 164; + value_165 = 165; + value_166 = 166; + value_167 = 167; + value_168 = 168; + value_169 = 169; + value_170 = 170; + value_171 = 171; + value_172 = 172; + value_173 = 173; + value_174 = 174; + value_175 = 175; + value_176 = 176; + value_177 = 177; + value_178 = 178; + value_179 = 179; + value_180 = 180; + value_181 = 181; + value_182 = 182; + value_183 = 183; + value_184 = 184; + value_185 = 185; + value_186 = 186; + value_187 = 187; + value_188 = 188; + value_189 = 189; + value_190 = 190; + value_191 = 191; + value_192 = 192; + value_193 = 193; + value_194 = 194; + value_195 = 195; + value_196 = 196; + value_197 = 197; + value_198 = 198; + value_199 = 199; + value_200 = 200; + value_201 = 201; + value_202 = 202; + value_203 = 203; + value_204 = 204; + value_205 = 205; + value_206 = 206; + value_207 = 207; + value_208 = 208; + value_209 = 209; + value_210 = 210; + value_211 = 211; + value_212 = 212; + value_213 = 213; + value_214 = 214; + value_215 = 215; + value_216 = 216; + value_217 = 217; + value_218 = 218; + value_219 = 219; + value_220 = 220; + value_221 = 221; + value_222 = 222; + value_223 = 223; + value_224 = 224; + value_225 = 225; + value_226 = 226; + value_227 = 227; + value_228 = 228; + value_229 = 229; + value_230 = 230; + value_231 = 231; + value_232 = 232; + value_233 = 233; + value_234 = 234; + value_235 = 235; + value_236 = 236; + value_237 = 237; + value_238 = 238; + value_239 = 239; + value_240 = 240; + value_241 = 241; + value_242 = 242; + value_243 = 243; + value_244 = 244; + value_245 = 245; + value_246 = 246; + value_247 = 247; + value_248 = 248; + value_249 = 249; + value_250 = 250; + value_251 = 251; + value_252 = 252; + value_253 = 253; + value_254 = 254; + value_255 = 255; + value_256 = 256; + value_257 = 257; + value_258 = 258; + value_259 = 259; + value_260 = 260; + value_261 = 261; + value_262 = 262; + value_263 = 263; + value_264 = 264; + value_265 = 265; + value_266 = 266; + value_267 = 267; + value_268 = 268; + value_269 = 269; + value_270 = 270; + value_271 = 271; + value_272 = 272; + value_273 = 273; + value_274 = 274; + value_275 = 275; + value_276 = 276; + value_277 = 277; + value_278 = 278; + value_279 = 279; + value_280 = 280; + value_281 = 281; + value_282 = 282; + value_283 = 283; + value_284 = 284; + value_285 = 285; + value_286 = 286; + value_287 = 287; + value_288 = 288; + value_289 = 289; + value_290 = 290; + value_291 = 291; + value_292 = 292; + value_293 = 293; + value_294 = 294; + value_295 = 295; + value_296 = 296; + value_297 = 297; + value_298 = 298; + value_299 = 299; + value_300 = 300; + value_301 = 301; + value_302 = 302; + value_303 = 303; + value_304 = 304; + value_305 = 305; + value_306 = 306; + value_307 = 307; + value_308 = 308; + value_309 = 309; + value_310 = 310; + value_311 = 311; + value_312 = 312; + value_313 = 313; + value_314 = 314; + value_315 = 315; + value_316 = 316; + value_317 = 317; + value_318 = 318; + value_319 = 319; + value_320 = 320; + value_321 = 321; + value_322 = 322; + value_323 = 323; + value_324 = 324; + value_325 = 325; + value_326 = 326; + value_327 = 327; + value_328 = 328; + value_329 = 329; + value_330 = 330; + value_331 = 331; + value_332 = 332; + value_333 = 333; + value_334 = 334; + value_335 = 335; + value_336 = 336; + value_337 = 337; + value_338 = 338; + value_339 = 339; + value_340 = 340; + value_341 = 341; + value_342 = 342; + value_343 = 343; + value_344 = 344; + value_345 = 345; + value_346 = 346; + value_347 = 347; + value_348 = 348; + value_349 = 349; + value_350 = 350; + value_351 = 351; + value_352 = 352; + value_353 = 353; + value_354 = 354; + value_355 = 355; + value_356 = 356; + value_357 = 357; + value_358 = 358; + value_359 = 359; + value_360 = 360; + value_361 = 361; + value_362 = 362; + value_363 = 363; + value_364 = 364; + value_365 = 365; + value_366 = 366; + value_367 = 367; + value_368 = 368; + value_369 = 369; + value_370 = 370; + value_371 = 371; + value_372 = 372; + value_373 = 373; + value_374 = 374; + value_375 = 375; + value_376 = 376; + value_377 = 377; + value_378 = 378; + value_379 = 379; + value_380 = 380; + value_381 = 381; + value_382 = 382; + value_383 = 383; + value_384 = 384; + value_385 = 385; + value_386 = 386; + value_387 = 387; + value_388 = 388; + value_389 = 389; + value_390 = 390; + value_391 = 391; + value_392 = 392; + value_393 = 393; + value_394 = 394; + value_395 = 395; + value_396 = 396; + value_397 = 397; + value_398 = 398; + value_399 = 399; + value_400 = 400; + value_401 = 401; + value_402 = 402; + value_403 = 403; + value_404 = 404; + value_405 = 405; + value_406 = 406; + value_407 = 407; + value_408 = 408; + value_409 = 409; + value_410 = 410; + value_411 = 411; + value_412 = 412; + value_413 = 413; + value_414 = 414; + value_415 = 415; + value_416 = 416; + value_417 = 417; + value_418 = 418; + value_419 = 419; + value_420 = 420; + value_421 = 421; + value_422 = 422; + value_423 = 423; + value_424 = 424; + value_425 = 425; + value_426 = 426; + value_427 = 427; + value_428 = 428; + value_429 = 429; + value_430 = 430; + value_431 = 431; + value_432 = 432; + value_433 = 433; + value_434 = 434; + value_435 = 435; + value_436 = 436; + value_437 = 437; + value_438 = 438; + value_439 = 439; + value_440 = 440; + value_441 = 441; + value_442 = 442; + value_443 = 443; + value_444 = 444; + value_445 = 445; + value_446 = 446; + value_447 = 447; + value_448 = 448; + value_449 = 449; + value_450 = 450; + value_451 = 451; + value_452 = 452; + value_453 = 453; + value_454 = 454; + value_455 = 455; + value_456 = 456; + value_457 = 457; + value_458 = 458; + value_459 = 459; + value_460 = 460; + value_461 = 461; + value_462 = 462; + value_463 = 463; + value_464 = 464; + value_465 = 465; + value_466 = 466; + value_467 = 467; + value_468 = 468; + value_469 = 469; + value_470 = 470; + value_471 = 471; + value_472 = 472; + value_473 = 473; + value_474 = 474; + value_475 = 475; + value_476 = 476; + value_477 = 477; + value_478 = 478; + value_479 = 479; + value_480 = 480; + value_481 = 481; + value_482 = 482; + value_483 = 483; + value_484 = 484; + value_485 = 485; + value_486 = 486; + value_487 = 487; + value_488 = 488; + value_489 = 489; + value_490 = 490; + value_491 = 491; + value_492 = 492; + value_493 = 493; + value_494 = 494; + value_495 = 495; + value_496 = 496; + value_497 = 497; + value_498 = 498; + value_499 = 499; + value_500 = 500; + value_501 = 501; + value_502 = 502; + value_503 = 503; + value_504 = 504; + value_505 = 505; + value_506 = 506; + value_507 = 507; + } + Value value = 1; + Sentence inner = 2; +} +message Sentence { + repeated Word words = 1; +} \ No newline at end of file diff --git a/src/Parsers/fuzzers/codegen_fuzzer/update.sh b/src/Parsers/fuzzers/codegen_fuzzer/update.sh new file mode 100755 index 00000000000..0982d6d0686 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/update.sh @@ -0,0 +1,30 @@ +#!/bin/bash + + +_main() { + local dict_filename="${1}" + if [[ $# -ne 1 ]]; + then + echo "Usage: $0 "; + exit 1; + fi + + if [[ ! -f $dict_filename ]]; + then + echo "File $dict_filename doesn't exist"; + exit 1 + fi + + cat clickhouse-template.g > clickhouse.g + + while read line; + do + [[ -z "$line" ]] && continue + echo $line | sed -e 's/"\(.*\)"/" \1 ";/g' + done < $dict_filename >> clickhouse.g +} + +_main "$@" + +# Sample run: ./update.sh ../../../../tests/fuzz/ast.dict +# then run `python ./gen.py clickhouse.g out.cpp out.proto` to generate new files with tokens. Rebuild fuzzer From 9ed33612980c59ac38ea63182407071148befd39 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Tue, 12 Oct 2021 03:53:43 +0300 Subject: [PATCH 11/89] removing code generated files, since protobuf generation is now done in CMake --- src/Parsers/fuzzers/codegen_fuzzer/out.cpp | 2189 ------------------ src/Parsers/fuzzers/codegen_fuzzer/out.proto | 519 ----- 2 files changed, 2708 deletions(-) delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.cpp delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.proto diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp deleted file mode 100644 index effe6e7821b..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp +++ /dev/null @@ -1,2189 +0,0 @@ -#include -#include -#include - -#include - -#include "out.pb.h" - -void GenerateWord(const Word&, std::string&, int); - -void GenerateSentence(const Sentence& stc, std::string &s, int depth) { - for (int i = 0; i < stc.words_size(); i++ ) { - GenerateWord(stc.words(i), s, ++depth); - } -} -void GenerateWord(const Word& word, std::string &s, int depth) { - if (depth > 5) return; - - switch (word.value()) { - case 0: { - s += " "; - break; - } - case 1: { - s += " "; - break; - } - case 2: { - s += " "; - break; - } - case 3: { - s += ";"; - break; - } - case 4: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ")"; - break; - } - case 5: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ")"; - break; - } - case 6: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ")"; - break; - } - case 7: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 8: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 9: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 10: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ", "; - if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); - break; - } - case 11: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "]"; - break; - } - case 12: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += "]"; - break; - } - case 13: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += "]"; - break; - } - case 14: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ", "; - if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); - s += "]"; - break; - } - case 15: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ")"; - break; - } - case 16: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ")"; - break; - } - case 17: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ")"; - break; - } - case 18: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " as "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 19: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " WHERE "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 20: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " GROUP BY "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 21: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " SORT BY "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 22: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " LIMIT "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 23: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 24: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " ARRAY JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 25: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += " ON "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 26: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += " USING "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 27: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " INTO OUTFILE "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 28: { - s += "WITH "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " AS "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 29: { - s += "{"; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ":"; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "}"; - break; - } - case 30: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ","; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "]"; - break; - } - case 31: { - s += "[]"; - break; - } - case 32: { - s += " x "; - break; - } - case 33: { - s += "x"; - break; - } - case 34: { - s += " `x` "; - break; - } - case 35: { - s += "`x`"; - break; - } - case 36: { - s += " \"value\" "; - break; - } - case 37: { - s += "\"value\""; - break; - } - case 38: { - s += " 0 "; - break; - } - case 39: { - s += "0"; - break; - } - case 40: { - s += "1"; - break; - } - case 41: { - s += "2"; - break; - } - case 42: { - s += "123123123123123123"; - break; - } - case 43: { - s += "182374019873401982734091873420923123123123123123"; - break; - } - case 44: { - s += "1e-1"; - break; - } - case 45: { - s += "1.1"; - break; - } - case 46: { - s += "\"\""; - break; - } - case 47: { - s += " '../../../../../../../../../etc/passwd' "; - break; - } - case 48: { - s += "/"; - break; - } - case 49: { - s += "="; - break; - } - case 50: { - s += "=="; - break; - } - case 51: { - s += "!="; - break; - } - case 52: { - s += "<>"; - break; - } - case 53: { - s += "<"; - break; - } - case 54: { - s += "<="; - break; - } - case 55: { - s += ">"; - break; - } - case 56: { - s += ">="; - break; - } - case 57: { - s += "<<"; - break; - } - case 58: { - s += "|<<"; - break; - } - case 59: { - s += "&"; - break; - } - case 60: { - s += "|"; - break; - } - case 61: { - s += "||"; - break; - } - case 62: { - s += "<|"; - break; - } - case 63: { - s += "|>"; - break; - } - case 64: { - s += "+"; - break; - } - case 65: { - s += "-"; - break; - } - case 66: { - s += "~"; - break; - } - case 67: { - s += "*"; - break; - } - case 68: { - s += "/"; - break; - } - case 69: { - s += "\\"; - break; - } - case 70: { - s += "%"; - break; - } - case 71: { - s += ""; - break; - } - case 72: { - s += "."; - break; - } - case 73: { - s += ","; - break; - } - case 74: { - s += ","; - break; - } - case 75: { - s += ","; - break; - } - case 76: { - s += ","; - break; - } - case 77: { - s += ","; - break; - } - case 78: { - s += ","; - break; - } - case 79: { - s += "("; - break; - } - case 80: { - s += ")"; - break; - } - case 81: { - s += "("; - break; - } - case 82: { - s += ")"; - break; - } - case 83: { - s += "("; - break; - } - case 84: { - s += ")"; - break; - } - case 85: { - s += "("; - break; - } - case 86: { - s += ")"; - break; - } - case 87: { - s += "("; - break; - } - case 88: { - s += ")"; - break; - } - case 89: { - s += "("; - break; - } - case 90: { - s += ")"; - break; - } - case 91: { - s += "?"; - break; - } - case 92: { - s += ":"; - break; - } - case 93: { - s += "@"; - break; - } - case 94: { - s += "@@"; - break; - } - case 95: { - s += "$"; - break; - } - case 96: { - s += "\""; - break; - } - case 97: { - s += "`"; - break; - } - case 98: { - s += "{"; - break; - } - case 99: { - s += "}"; - break; - } - case 100: { - s += "^"; - break; - } - case 101: { - s += "::"; - break; - } - case 102: { - s += "->"; - break; - } - case 103: { - s += "]"; - break; - } - case 104: { - s += "["; - break; - } - case 105: { - s += " ADD "; - break; - } - case 106: { - s += " ADD COLUMN "; - break; - } - case 107: { - s += " ADD CONSTRAINT "; - break; - } - case 108: { - s += " ADD INDEX "; - break; - } - case 109: { - s += " AFTER "; - break; - } - case 110: { - s += " AggregateFunction "; - break; - } - case 111: { - s += " aggThrow "; - break; - } - case 112: { - s += " ALL "; - break; - } - case 113: { - s += " ALTER LIVE VIEW "; - break; - } - case 114: { - s += " ALTER TABLE "; - break; - } - case 115: { - s += " and "; - break; - } - case 116: { - s += " ANTI "; - break; - } - case 117: { - s += " any "; - break; - } - case 118: { - s += " anyHeavy "; - break; - } - case 119: { - s += " anyLast "; - break; - } - case 120: { - s += " argMax "; - break; - } - case 121: { - s += " argMin "; - break; - } - case 122: { - s += " array "; - break; - } - case 123: { - s += " Array "; - break; - } - case 124: { - s += " arrayAll "; - break; - } - case 125: { - s += " arrayAUC "; - break; - } - case 126: { - s += " arrayCompact "; - break; - } - case 127: { - s += " arrayConcat "; - break; - } - case 128: { - s += " arrayCount "; - break; - } - case 129: { - s += " arrayCumSum "; - break; - } - case 130: { - s += " arrayCumSumNonNegative "; - break; - } - case 131: { - s += " arrayDifference "; - break; - } - case 132: { - s += " arrayDistinct "; - break; - } - case 133: { - s += " arrayElement "; - break; - } - case 134: { - s += " arrayEnumerate "; - break; - } - case 135: { - s += " arrayEnumerateDense "; - break; - } - case 136: { - s += " arrayEnumerateDenseRanked "; - break; - } - case 137: { - s += " arrayEnumerateUniq "; - break; - } - case 138: { - s += " arrayEnumerateUniqRanked "; - break; - } - case 139: { - s += " arrayExists "; - break; - } - case 140: { - s += " arrayFill "; - break; - } - case 141: { - s += " arrayFilter "; - break; - } - case 142: { - s += " arrayFirst "; - break; - } - case 143: { - s += " arrayFirstIndex "; - break; - } - case 144: { - s += " arrayFlatten "; - break; - } - case 145: { - s += " arrayIntersect "; - break; - } - case 146: { - s += " arrayJoin "; - break; - } - case 147: { - s += " ARRAY JOIN "; - break; - } - case 148: { - s += " arrayMap "; - break; - } - case 149: { - s += " arrayPopBack "; - break; - } - case 150: { - s += " arrayPopFront "; - break; - } - case 151: { - s += " arrayPushBack "; - break; - } - case 152: { - s += " arrayPushFront "; - break; - } - case 153: { - s += " arrayReduce "; - break; - } - case 154: { - s += " arrayReduceInRanges "; - break; - } - case 155: { - s += " arrayResize "; - break; - } - case 156: { - s += " arrayReverse "; - break; - } - case 157: { - s += " arrayReverseFill "; - break; - } - case 158: { - s += " arrayReverseSort "; - break; - } - case 159: { - s += " arrayReverseSplit "; - break; - } - case 160: { - s += " arraySlice "; - break; - } - case 161: { - s += " arraySort "; - break; - } - case 162: { - s += " arraySplit "; - break; - } - case 163: { - s += " arraySum "; - break; - } - case 164: { - s += " arrayUniq "; - break; - } - case 165: { - s += " arrayWithConstant "; - break; - } - case 166: { - s += " arrayZip "; - break; - } - case 167: { - s += " AS "; - break; - } - case 168: { - s += " ASC "; - break; - } - case 169: { - s += " ASCENDING "; - break; - } - case 170: { - s += " ASOF "; - break; - } - case 171: { - s += " AST "; - break; - } - case 172: { - s += " ATTACH "; - break; - } - case 173: { - s += " ATTACH PART "; - break; - } - case 174: { - s += " ATTACH PARTITION "; - break; - } - case 175: { - s += " avg "; - break; - } - case 176: { - s += " avgWeighted "; - break; - } - case 177: { - s += " basename "; - break; - } - case 178: { - s += " BETWEEN "; - break; - } - case 179: { - s += " BOTH "; - break; - } - case 180: { - s += " boundingRatio "; - break; - } - case 181: { - s += " BY "; - break; - } - case 182: { - s += " CAST "; - break; - } - case 183: { - s += " categoricalInformationValue "; - break; - } - case 184: { - s += " CHECK "; - break; - } - case 185: { - s += " CHECK TABLE "; - break; - } - case 186: { - s += " CLEAR COLUMN "; - break; - } - case 187: { - s += " CLEAR INDEX "; - break; - } - case 188: { - s += " COLLATE "; - break; - } - case 189: { - s += " COLUMNS "; - break; - } - case 190: { - s += " COMMENT COLUMN "; - break; - } - case 191: { - s += " CONSTRAINT "; - break; - } - case 192: { - s += " corr "; - break; - } - case 193: { - s += " corrStable "; - break; - } - case 194: { - s += " count "; - break; - } - case 195: { - s += " countEqual "; - break; - } - case 196: { - s += " covarPop "; - break; - } - case 197: { - s += " covarPopStable "; - break; - } - case 198: { - s += " covarSamp "; - break; - } - case 199: { - s += " covarSampStable "; - break; - } - case 200: { - s += " CREATE "; - break; - } - case 201: { - s += " CROSS "; - break; - } - case 202: { - s += " CUBE "; - break; - } - case 203: { - s += " cutFragment "; - break; - } - case 204: { - s += " cutQueryString "; - break; - } - case 205: { - s += " cutQueryStringAndFragment "; - break; - } - case 206: { - s += " cutToFirstSignificantSubdomain "; - break; - } - case 207: { - s += " cutURLParameter "; - break; - } - case 208: { - s += " cutWWW "; - break; - } - case 209: { - s += " D "; - break; - } - case 210: { - s += " DATABASE "; - break; - } - case 211: { - s += " DATABASES "; - break; - } - case 212: { - s += " Date "; - break; - } - case 213: { - s += " DATE "; - break; - } - case 214: { - s += " DATE_ADD "; - break; - } - case 215: { - s += " DATEADD "; - break; - } - case 216: { - s += " DATE_DIFF "; - break; - } - case 217: { - s += " DATEDIFF "; - break; - } - case 218: { - s += " DATE_SUB "; - break; - } - case 219: { - s += " DATESUB "; - break; - } - case 220: { - s += " DateTime "; - break; - } - case 221: { - s += " DateTime64 "; - break; - } - case 222: { - s += " DAY "; - break; - } - case 223: { - s += " DD "; - break; - } - case 224: { - s += " Decimal "; - break; - } - case 225: { - s += " Decimal128 "; - break; - } - case 226: { - s += " Decimal32 "; - break; - } - case 227: { - s += " Decimal64 "; - break; - } - case 228: { - s += " decodeURLComponent "; - break; - } - case 229: { - s += " DEDUPLICATE "; - break; - } - case 230: { - s += " DELETE "; - break; - } - case 231: { - s += " DELETE WHERE "; - break; - } - case 232: { - s += " DESC "; - break; - } - case 233: { - s += " DESCENDING "; - break; - } - case 234: { - s += " DESCRIBE "; - break; - } - case 235: { - s += " DETACH "; - break; - } - case 236: { - s += " DETACH PARTITION "; - break; - } - case 237: { - s += " DICTIONARIES "; - break; - } - case 238: { - s += " DICTIONARY "; - break; - } - case 239: { - s += " DISTINCT "; - break; - } - case 240: { - s += " domain "; - break; - } - case 241: { - s += " domainWithoutWWW "; - break; - } - case 242: { - s += " DROP "; - break; - } - case 243: { - s += " DROP COLUMN "; - break; - } - case 244: { - s += " DROP CONSTRAINT "; - break; - } - case 245: { - s += " DROP DETACHED PART "; - break; - } - case 246: { - s += " DROP DETACHED PARTITION "; - break; - } - case 247: { - s += " DROP INDEX "; - break; - } - case 248: { - s += " DROP PARTITION "; - break; - } - case 249: { - s += " emptyArrayToSingle "; - break; - } - case 250: { - s += " ENGINE "; - break; - } - case 251: { - s += " entropy "; - break; - } - case 252: { - s += " Enum "; - break; - } - case 253: { - s += " Enum16 "; - break; - } - case 254: { - s += " Enum8 "; - break; - } - case 255: { - s += " EVENTS "; - break; - } - case 256: { - s += " EXCHANGE TABLES "; - break; - } - case 257: { - s += " EXISTS "; - break; - } - case 258: { - s += " EXTRACT "; - break; - } - case 259: { - s += " extractURLParameter "; - break; - } - case 260: { - s += " extractURLParameterNames "; - break; - } - case 261: { - s += " extractURLParameters "; - break; - } - case 262: { - s += " FETCH PARTITION "; - break; - } - case 263: { - s += " FETCH PART "; - break; - } - case 264: { - s += " FINAL "; - break; - } - case 265: { - s += " FIRST "; - break; - } - case 266: { - s += " firstSignificantSubdomain "; - break; - } - case 267: { - s += " FixedString "; - break; - } - case 268: { - s += " Float32 "; - break; - } - case 269: { - s += " Float64 "; - break; - } - case 270: { - s += " FOR "; - break; - } - case 271: { - s += " ForEach "; - break; - } - case 272: { - s += " FORMAT "; - break; - } - case 273: { - s += " fragment "; - break; - } - case 274: { - s += " FREEZE "; - break; - } - case 275: { - s += " FROM "; - break; - } - case 276: { - s += " FULL "; - break; - } - case 277: { - s += " FUNCTION "; - break; - } - case 278: { - s += " __getScalar "; - break; - } - case 279: { - s += " GLOBAL "; - break; - } - case 280: { - s += " GRANULARITY "; - break; - } - case 281: { - s += " groupArray "; - break; - } - case 282: { - s += " groupArrayInsertAt "; - break; - } - case 283: { - s += " groupArrayMovingAvg "; - break; - } - case 284: { - s += " groupArrayMovingSum "; - break; - } - case 285: { - s += " groupArraySample "; - break; - } - case 286: { - s += " groupBitAnd "; - break; - } - case 287: { - s += " groupBitmap "; - break; - } - case 288: { - s += " groupBitmapAnd "; - break; - } - case 289: { - s += " groupBitmapOr "; - break; - } - case 290: { - s += " groupBitmapXor "; - break; - } - case 291: { - s += " groupBitOr "; - break; - } - case 292: { - s += " groupBitXor "; - break; - } - case 293: { - s += " GROUP BY "; - break; - } - case 294: { - s += " groupUniqArray "; - break; - } - case 295: { - s += " has "; - break; - } - case 296: { - s += " hasAll "; - break; - } - case 297: { - s += " hasAny "; - break; - } - case 298: { - s += " HAVING "; - break; - } - case 299: { - s += " HH "; - break; - } - case 300: { - s += " histogram "; - break; - } - case 301: { - s += " HOUR "; - break; - } - case 302: { - s += " ID "; - break; - } - case 303: { - s += " if "; - break; - } - case 304: { - s += " IF EXISTS "; - break; - } - case 305: { - s += " IF NOT EXISTS "; - break; - } - case 306: { - s += " IN "; - break; - } - case 307: { - s += " INDEX "; - break; - } - case 308: { - s += " indexOf "; - break; - } - case 309: { - s += " INNER "; - break; - } - case 310: { - s += " IN PARTITION "; - break; - } - case 311: { - s += " INSERT INTO "; - break; - } - case 312: { - s += " Int16 "; - break; - } - case 313: { - s += " Int32 "; - break; - } - case 314: { - s += " Int64 "; - break; - } - case 315: { - s += " Int8 "; - break; - } - case 316: { - s += " INTERVAL "; - break; - } - case 317: { - s += " IntervalDay "; - break; - } - case 318: { - s += " IntervalHour "; - break; - } - case 319: { - s += " IntervalMinute "; - break; - } - case 320: { - s += " IntervalMonth "; - break; - } - case 321: { - s += " IntervalQuarter "; - break; - } - case 322: { - s += " IntervalSecond "; - break; - } - case 323: { - s += " IntervalWeek "; - break; - } - case 324: { - s += " IntervalYear "; - break; - } - case 325: { - s += " INTO OUTFILE "; - break; - } - case 326: { - s += " JOIN "; - break; - } - case 327: { - s += " kurtPop "; - break; - } - case 328: { - s += " kurtSamp "; - break; - } - case 329: { - s += " LAST "; - break; - } - case 330: { - s += " LAYOUT "; - break; - } - case 331: { - s += " LEADING "; - break; - } - case 332: { - s += " LEFT "; - break; - } - case 333: { - s += " LEFT ARRAY JOIN "; - break; - } - case 334: { - s += " length "; - break; - } - case 335: { - s += " LIFETIME "; - break; - } - case 336: { - s += " LIKE "; - break; - } - case 337: { - s += " LIMIT "; - break; - } - case 338: { - s += " LIVE "; - break; - } - case 339: { - s += " LOCAL "; - break; - } - case 340: { - s += " LowCardinality "; - break; - } - case 341: { - s += " LTRIM "; - break; - } - case 342: { - s += " M "; - break; - } - case 343: { - s += " MATERIALIZED "; - break; - } - case 344: { - s += " MATERIALIZE INDEX "; - break; - } - case 345: { - s += " MATERIALIZE TTL "; - break; - } - case 346: { - s += " max "; - break; - } - case 347: { - s += " maxIntersections "; - break; - } - case 348: { - s += " maxIntersectionsPosition "; - break; - } - case 349: { - s += " Merge "; - break; - } - case 350: { - s += " MI "; - break; - } - case 351: { - s += " min "; - break; - } - case 352: { - s += " MINUTE "; - break; - } - case 353: { - s += " MM "; - break; - } - case 354: { - s += " MODIFY "; - break; - } - case 355: { - s += " MODIFY COLUMN "; - break; - } - case 356: { - s += " MODIFY ORDER BY "; - break; - } - case 357: { - s += " MODIFY QUERY "; - break; - } - case 358: { - s += " MODIFY SETTING "; - break; - } - case 359: { - s += " MODIFY TTL "; - break; - } - case 360: { - s += " MONTH "; - break; - } - case 361: { - s += " MOVE PART "; - break; - } - case 362: { - s += " MOVE PARTITION "; - break; - } - case 363: { - s += " movingXXX "; - break; - } - case 364: { - s += " N "; - break; - } - case 365: { - s += " NAME "; - break; - } - case 366: { - s += " Nested "; - break; - } - case 367: { - s += " NO DELAY "; - break; - } - case 368: { - s += " NONE "; - break; - } - case 369: { - s += " not "; - break; - } - case 370: { - s += " nothing "; - break; - } - case 371: { - s += " Nothing "; - break; - } - case 372: { - s += " Null "; - break; - } - case 373: { - s += " Nullable "; - break; - } - case 374: { - s += " NULLS "; - break; - } - case 375: { - s += " OFFSET "; - break; - } - case 376: { - s += " ON "; - break; - } - case 377: { - s += " ONLY "; - break; - } - case 378: { - s += " OPTIMIZE TABLE "; - break; - } - case 379: { - s += " ORDER BY "; - break; - } - case 380: { - s += " OR REPLACE "; - break; - } - case 381: { - s += " OUTER "; - break; - } - case 382: { - s += " PARTITION "; - break; - } - case 383: { - s += " PARTITION BY "; - break; - } - case 384: { - s += " path "; - break; - } - case 385: { - s += " pathFull "; - break; - } - case 386: { - s += " POPULATE "; - break; - } - case 387: { - s += " PREWHERE "; - break; - } - case 388: { - s += " PRIMARY KEY "; - break; - } - case 389: { - s += " protocol "; - break; - } - case 390: { - s += " Q "; - break; - } - case 391: { - s += " QQ "; - break; - } - case 392: { - s += " QUARTER "; - break; - } - case 393: { - s += " queryString "; - break; - } - case 394: { - s += " queryStringAndFragment "; - break; - } - case 395: { - s += " range "; - break; - } - case 396: { - s += " REFRESH "; - break; - } - case 397: { - s += " RENAME COLUMN "; - break; - } - case 398: { - s += " RENAME TABLE "; - break; - } - case 399: { - s += " REPLACE PARTITION "; - break; - } - case 400: { - s += " Resample "; - break; - } - case 401: { - s += " RESUME "; - break; - } - case 402: { - s += " retention "; - break; - } - case 403: { - s += " RIGHT "; - break; - } - case 404: { - s += " ROLLUP "; - break; - } - case 405: { - s += " RTRIM "; - break; - } - case 406: { - s += " S "; - break; - } - case 407: { - s += " SAMPLE "; - break; - } - case 408: { - s += " SAMPLE BY "; - break; - } - case 409: { - s += " SECOND "; - break; - } - case 410: { - s += " SELECT "; - break; - } - case 411: { - s += " SEMI "; - break; - } - case 412: { - s += " sequenceCount "; - break; - } - case 413: { - s += " sequenceMatch "; - break; - } - case 414: { - s += " SET "; - break; - } - case 415: { - s += " SETTINGS "; - break; - } - case 416: { - s += " SHOW "; - break; - } - case 417: { - s += " SHOW PROCESSLIST "; - break; - } - case 418: { - s += " simpleLinearRegression "; - break; - } - case 419: { - s += " skewPop "; - break; - } - case 420: { - s += " skewSamp "; - break; - } - case 421: { - s += " SOURCE "; - break; - } - case 422: { - s += " SQL_TSI_DAY "; - break; - } - case 423: { - s += " SQL_TSI_HOUR "; - break; - } - case 424: { - s += " SQL_TSI_MINUTE "; - break; - } - case 425: { - s += " SQL_TSI_MONTH "; - break; - } - case 426: { - s += " SQL_TSI_QUARTER "; - break; - } - case 427: { - s += " SQL_TSI_SECOND "; - break; - } - case 428: { - s += " SQL_TSI_WEEK "; - break; - } - case 429: { - s += " SQL_TSI_YEAR "; - break; - } - case 430: { - s += " SS "; - break; - } - case 431: { - s += " State "; - break; - } - case 432: { - s += " stddevPop "; - break; - } - case 433: { - s += " stddevPopStable "; - break; - } - case 434: { - s += " stddevSamp "; - break; - } - case 435: { - s += " stddevSampStable "; - break; - } - case 436: { - s += " STEP "; - break; - } - case 437: { - s += " stochasticLinearRegression "; - break; - } - case 438: { - s += " stochasticLogisticRegression "; - break; - } - case 439: { - s += " String "; - break; - } - case 440: { - s += " SUBSTRING "; - break; - } - case 441: { - s += " sum "; - break; - } - case 442: { - s += " sumKahan "; - break; - } - case 443: { - s += " sumMap "; - break; - } - case 444: { - s += " sumMapFiltered "; - break; - } - case 445: { - s += " sumMapFilteredWithOverflow "; - break; - } - case 446: { - s += " sumMapWithOverflow "; - break; - } - case 447: { - s += " sumWithOverflow "; - break; - } - case 448: { - s += " SUSPEND "; - break; - } - case 449: { - s += " TABLE "; - break; - } - case 450: { - s += " TABLES "; - break; - } - case 451: { - s += " TEMPORARY "; - break; - } - case 452: { - s += " TIMESTAMP "; - break; - } - case 453: { - s += " TIMESTAMP_ADD "; - break; - } - case 454: { - s += " TIMESTAMPADD "; - break; - } - case 455: { - s += " TIMESTAMP_DIFF "; - break; - } - case 456: { - s += " TIMESTAMPDIFF "; - break; - } - case 457: { - s += " TIMESTAMP_SUB "; - break; - } - case 458: { - s += " TIMESTAMPSUB "; - break; - } - case 459: { - s += " TO "; - break; - } - case 460: { - s += " TO DISK "; - break; - } - case 461: { - s += " TOP "; - break; - } - case 462: { - s += " topK "; - break; - } - case 463: { - s += " topKWeighted "; - break; - } - case 464: { - s += " topLevelDomain "; - break; - } - case 465: { - s += " TO TABLE "; - break; - } - case 466: { - s += " TOTALS "; - break; - } - case 467: { - s += " TO VOLUME "; - break; - } - case 468: { - s += " TRAILING "; - break; - } - case 469: { - s += " TRIM "; - break; - } - case 470: { - s += " TRUNCATE "; - break; - } - case 471: { - s += " TTL "; - break; - } - case 472: { - s += " Tuple "; - break; - } - case 473: { - s += " TYPE "; - break; - } - case 474: { - s += " UInt16 "; - break; - } - case 475: { - s += " UInt32 "; - break; - } - case 476: { - s += " UInt64 "; - break; - } - case 477: { - s += " UInt8 "; - break; - } - case 478: { - s += " uniq "; - break; - } - case 479: { - s += " uniqCombined "; - break; - } - case 480: { - s += " uniqCombined64 "; - break; - } - case 481: { - s += " uniqExact "; - break; - } - case 482: { - s += " uniqHLL12 "; - break; - } - case 483: { - s += " uniqUpTo "; - break; - } - case 484: { - s += " UPDATE "; - break; - } - case 485: { - s += " URLHierarchy "; - break; - } - case 486: { - s += " URLPathHierarchy "; - break; - } - case 487: { - s += " USE "; - break; - } - case 488: { - s += " USING "; - break; - } - case 489: { - s += " UUID "; - break; - } - case 490: { - s += " VALUES "; - break; - } - case 491: { - s += " varPop "; - break; - } - case 492: { - s += " varPopStable "; - break; - } - case 493: { - s += " varSamp "; - break; - } - case 494: { - s += " varSampStable "; - break; - } - case 495: { - s += " VIEW "; - break; - } - case 496: { - s += " WATCH "; - break; - } - case 497: { - s += " WEEK "; - break; - } - case 498: { - s += " WHERE "; - break; - } - case 499: { - s += " windowFunnel "; - break; - } - case 500: { - s += " WITH "; - break; - } - case 501: { - s += " WITH FILL "; - break; - } - case 502: { - s += " WITH TIES "; - break; - } - case 503: { - s += " WK "; - break; - } - case 504: { - s += " WW "; - break; - } - case 505: { - s += " YEAR "; - break; - } - case 506: { - s += " YY "; - break; - } - case 507: { - s += " YYYY "; - break; - } - default: break; - } -} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.proto b/src/Parsers/fuzzers/codegen_fuzzer/out.proto deleted file mode 100644 index 6c8cefce9a3..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/out.proto +++ /dev/null @@ -1,519 +0,0 @@ -syntax = "proto3"; - -message Word { - enum Value { - value_0 = 0; - value_1 = 1; - value_2 = 2; - value_3 = 3; - value_4 = 4; - value_5 = 5; - value_6 = 6; - value_7 = 7; - value_8 = 8; - value_9 = 9; - value_10 = 10; - value_11 = 11; - value_12 = 12; - value_13 = 13; - value_14 = 14; - value_15 = 15; - value_16 = 16; - value_17 = 17; - value_18 = 18; - value_19 = 19; - value_20 = 20; - value_21 = 21; - value_22 = 22; - value_23 = 23; - value_24 = 24; - value_25 = 25; - value_26 = 26; - value_27 = 27; - value_28 = 28; - value_29 = 29; - value_30 = 30; - value_31 = 31; - value_32 = 32; - value_33 = 33; - value_34 = 34; - value_35 = 35; - value_36 = 36; - value_37 = 37; - value_38 = 38; - value_39 = 39; - value_40 = 40; - value_41 = 41; - value_42 = 42; - value_43 = 43; - value_44 = 44; - value_45 = 45; - value_46 = 46; - value_47 = 47; - value_48 = 48; - value_49 = 49; - value_50 = 50; - value_51 = 51; - value_52 = 52; - value_53 = 53; - value_54 = 54; - value_55 = 55; - value_56 = 56; - value_57 = 57; - value_58 = 58; - value_59 = 59; - value_60 = 60; - value_61 = 61; - value_62 = 62; - value_63 = 63; - value_64 = 64; - value_65 = 65; - value_66 = 66; - value_67 = 67; - value_68 = 68; - value_69 = 69; - value_70 = 70; - value_71 = 71; - value_72 = 72; - value_73 = 73; - value_74 = 74; - value_75 = 75; - value_76 = 76; - value_77 = 77; - value_78 = 78; - value_79 = 79; - value_80 = 80; - value_81 = 81; - value_82 = 82; - value_83 = 83; - value_84 = 84; - value_85 = 85; - value_86 = 86; - value_87 = 87; - value_88 = 88; - value_89 = 89; - value_90 = 90; - value_91 = 91; - value_92 = 92; - value_93 = 93; - value_94 = 94; - value_95 = 95; - value_96 = 96; - value_97 = 97; - value_98 = 98; - value_99 = 99; - value_100 = 100; - value_101 = 101; - value_102 = 102; - value_103 = 103; - value_104 = 104; - value_105 = 105; - value_106 = 106; - value_107 = 107; - value_108 = 108; - value_109 = 109; - value_110 = 110; - value_111 = 111; - value_112 = 112; - value_113 = 113; - value_114 = 114; - value_115 = 115; - value_116 = 116; - value_117 = 117; - value_118 = 118; - value_119 = 119; - value_120 = 120; - value_121 = 121; - value_122 = 122; - value_123 = 123; - value_124 = 124; - value_125 = 125; - value_126 = 126; - value_127 = 127; - value_128 = 128; - value_129 = 129; - value_130 = 130; - value_131 = 131; - value_132 = 132; - value_133 = 133; - value_134 = 134; - value_135 = 135; - value_136 = 136; - value_137 = 137; - value_138 = 138; - value_139 = 139; - value_140 = 140; - value_141 = 141; - value_142 = 142; - value_143 = 143; - value_144 = 144; - value_145 = 145; - value_146 = 146; - value_147 = 147; - value_148 = 148; - value_149 = 149; - value_150 = 150; - value_151 = 151; - value_152 = 152; - value_153 = 153; - value_154 = 154; - value_155 = 155; - value_156 = 156; - value_157 = 157; - value_158 = 158; - value_159 = 159; - value_160 = 160; - value_161 = 161; - value_162 = 162; - value_163 = 163; - value_164 = 164; - value_165 = 165; - value_166 = 166; - value_167 = 167; - value_168 = 168; - value_169 = 169; - value_170 = 170; - value_171 = 171; - value_172 = 172; - value_173 = 173; - value_174 = 174; - value_175 = 175; - value_176 = 176; - value_177 = 177; - value_178 = 178; - value_179 = 179; - value_180 = 180; - value_181 = 181; - value_182 = 182; - value_183 = 183; - value_184 = 184; - value_185 = 185; - value_186 = 186; - value_187 = 187; - value_188 = 188; - value_189 = 189; - value_190 = 190; - value_191 = 191; - value_192 = 192; - value_193 = 193; - value_194 = 194; - value_195 = 195; - value_196 = 196; - value_197 = 197; - value_198 = 198; - value_199 = 199; - value_200 = 200; - value_201 = 201; - value_202 = 202; - value_203 = 203; - value_204 = 204; - value_205 = 205; - value_206 = 206; - value_207 = 207; - value_208 = 208; - value_209 = 209; - value_210 = 210; - value_211 = 211; - value_212 = 212; - value_213 = 213; - value_214 = 214; - value_215 = 215; - value_216 = 216; - value_217 = 217; - value_218 = 218; - value_219 = 219; - value_220 = 220; - value_221 = 221; - value_222 = 222; - value_223 = 223; - value_224 = 224; - value_225 = 225; - value_226 = 226; - value_227 = 227; - value_228 = 228; - value_229 = 229; - value_230 = 230; - value_231 = 231; - value_232 = 232; - value_233 = 233; - value_234 = 234; - value_235 = 235; - value_236 = 236; - value_237 = 237; - value_238 = 238; - value_239 = 239; - value_240 = 240; - value_241 = 241; - value_242 = 242; - value_243 = 243; - value_244 = 244; - value_245 = 245; - value_246 = 246; - value_247 = 247; - value_248 = 248; - value_249 = 249; - value_250 = 250; - value_251 = 251; - value_252 = 252; - value_253 = 253; - value_254 = 254; - value_255 = 255; - value_256 = 256; - value_257 = 257; - value_258 = 258; - value_259 = 259; - value_260 = 260; - value_261 = 261; - value_262 = 262; - value_263 = 263; - value_264 = 264; - value_265 = 265; - value_266 = 266; - value_267 = 267; - value_268 = 268; - value_269 = 269; - value_270 = 270; - value_271 = 271; - value_272 = 272; - value_273 = 273; - value_274 = 274; - value_275 = 275; - value_276 = 276; - value_277 = 277; - value_278 = 278; - value_279 = 279; - value_280 = 280; - value_281 = 281; - value_282 = 282; - value_283 = 283; - value_284 = 284; - value_285 = 285; - value_286 = 286; - value_287 = 287; - value_288 = 288; - value_289 = 289; - value_290 = 290; - value_291 = 291; - value_292 = 292; - value_293 = 293; - value_294 = 294; - value_295 = 295; - value_296 = 296; - value_297 = 297; - value_298 = 298; - value_299 = 299; - value_300 = 300; - value_301 = 301; - value_302 = 302; - value_303 = 303; - value_304 = 304; - value_305 = 305; - value_306 = 306; - value_307 = 307; - value_308 = 308; - value_309 = 309; - value_310 = 310; - value_311 = 311; - value_312 = 312; - value_313 = 313; - value_314 = 314; - value_315 = 315; - value_316 = 316; - value_317 = 317; - value_318 = 318; - value_319 = 319; - value_320 = 320; - value_321 = 321; - value_322 = 322; - value_323 = 323; - value_324 = 324; - value_325 = 325; - value_326 = 326; - value_327 = 327; - value_328 = 328; - value_329 = 329; - value_330 = 330; - value_331 = 331; - value_332 = 332; - value_333 = 333; - value_334 = 334; - value_335 = 335; - value_336 = 336; - value_337 = 337; - value_338 = 338; - value_339 = 339; - value_340 = 340; - value_341 = 341; - value_342 = 342; - value_343 = 343; - value_344 = 344; - value_345 = 345; - value_346 = 346; - value_347 = 347; - value_348 = 348; - value_349 = 349; - value_350 = 350; - value_351 = 351; - value_352 = 352; - value_353 = 353; - value_354 = 354; - value_355 = 355; - value_356 = 356; - value_357 = 357; - value_358 = 358; - value_359 = 359; - value_360 = 360; - value_361 = 361; - value_362 = 362; - value_363 = 363; - value_364 = 364; - value_365 = 365; - value_366 = 366; - value_367 = 367; - value_368 = 368; - value_369 = 369; - value_370 = 370; - value_371 = 371; - value_372 = 372; - value_373 = 373; - value_374 = 374; - value_375 = 375; - value_376 = 376; - value_377 = 377; - value_378 = 378; - value_379 = 379; - value_380 = 380; - value_381 = 381; - value_382 = 382; - value_383 = 383; - value_384 = 384; - value_385 = 385; - value_386 = 386; - value_387 = 387; - value_388 = 388; - value_389 = 389; - value_390 = 390; - value_391 = 391; - value_392 = 392; - value_393 = 393; - value_394 = 394; - value_395 = 395; - value_396 = 396; - value_397 = 397; - value_398 = 398; - value_399 = 399; - value_400 = 400; - value_401 = 401; - value_402 = 402; - value_403 = 403; - value_404 = 404; - value_405 = 405; - value_406 = 406; - value_407 = 407; - value_408 = 408; - value_409 = 409; - value_410 = 410; - value_411 = 411; - value_412 = 412; - value_413 = 413; - value_414 = 414; - value_415 = 415; - value_416 = 416; - value_417 = 417; - value_418 = 418; - value_419 = 419; - value_420 = 420; - value_421 = 421; - value_422 = 422; - value_423 = 423; - value_424 = 424; - value_425 = 425; - value_426 = 426; - value_427 = 427; - value_428 = 428; - value_429 = 429; - value_430 = 430; - value_431 = 431; - value_432 = 432; - value_433 = 433; - value_434 = 434; - value_435 = 435; - value_436 = 436; - value_437 = 437; - value_438 = 438; - value_439 = 439; - value_440 = 440; - value_441 = 441; - value_442 = 442; - value_443 = 443; - value_444 = 444; - value_445 = 445; - value_446 = 446; - value_447 = 447; - value_448 = 448; - value_449 = 449; - value_450 = 450; - value_451 = 451; - value_452 = 452; - value_453 = 453; - value_454 = 454; - value_455 = 455; - value_456 = 456; - value_457 = 457; - value_458 = 458; - value_459 = 459; - value_460 = 460; - value_461 = 461; - value_462 = 462; - value_463 = 463; - value_464 = 464; - value_465 = 465; - value_466 = 466; - value_467 = 467; - value_468 = 468; - value_469 = 469; - value_470 = 470; - value_471 = 471; - value_472 = 472; - value_473 = 473; - value_474 = 474; - value_475 = 475; - value_476 = 476; - value_477 = 477; - value_478 = 478; - value_479 = 479; - value_480 = 480; - value_481 = 481; - value_482 = 482; - value_483 = 483; - value_484 = 484; - value_485 = 485; - value_486 = 486; - value_487 = 487; - value_488 = 488; - value_489 = 489; - value_490 = 490; - value_491 = 491; - value_492 = 492; - value_493 = 493; - value_494 = 494; - value_495 = 495; - value_496 = 496; - value_497 = 497; - value_498 = 498; - value_499 = 499; - value_500 = 500; - value_501 = 501; - value_502 = 502; - value_503 = 503; - value_504 = 504; - value_505 = 505; - value_506 = 506; - value_507 = 507; - } - Value value = 1; - Sentence inner = 2; -} -message Sentence { - repeated Word words = 1; -} \ No newline at end of file From 3540baa33c2d04788f152edb862888e66492e14a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:58:33 +0300 Subject: [PATCH 12/89] Start server under gdb in functional tests --- docker/test/stateless/run.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ed721690281..ebb72111e96 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,6 +45,23 @@ else sudo clickhouse start fi +echo " +set follow-fork-mode child +handle all noprint +handle SIGSEGV stop print +handle SIGBUS stop print +handle SIGABRT stop print +continue +thread apply all backtrace +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From f1bbc7f9b633f5f3948358e574dad4d71c3f7199 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Oct 2021 23:17:15 +0300 Subject: [PATCH 13/89] Fix hardware utilization info printing in client --- docs/en/interfaces/cli.md | 1 + src/Client/ClientBase.cpp | 5 +++- src/Common/ProgressIndication.cpp | 41 +++++++++++++++++-------------- src/Common/ProgressIndication.h | 2 ++ 4 files changed, 30 insertions(+), 19 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 70b7d59b037..c4305d229cf 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -127,6 +127,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--secure` – If specified, will connect to server over secure connection. - `--history_file` — Path to a file containing command history. - `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). +- `--hardware-utilization` — Print hardware utilization information in progress bar. Since version 20.5, `clickhouse-client` has automatic syntax highlighting (always enabled). diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index baf082a3541..c93645a1f8a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -668,7 +668,7 @@ void ClientBase::onEndOfStream() void ClientBase::onProfileEvents(Block & block) { const auto rows = block.rows(); - if (rows == 0) + if (rows == 0 || !progress_indication.print_hardware_utilization) return; const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); @@ -1560,6 +1560,7 @@ void ClientBase::init(int argc, char ** argv) ("ignore-error", "do not stop processing in multiquery mode") ("stacktrace", "print stack traces of exceptions") + ("hardware-utilization", "print hardware utilization information in progress bar") ; addAndCheckOptions(options_description, options, common_arguments); @@ -1626,6 +1627,8 @@ void ClientBase::init(int argc, char ** argv) config().setBool("verbose", true); if (options.count("log-level")) Poco::Logger::root().setLevel(options["log-level"].as()); + if (options.count("hardware-utilization")) + progress_indication.print_hardware_utilization = true; query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 0fe40b306cb..4510952cc71 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "IO/WriteBufferFromString.h" #include @@ -189,6 +190,26 @@ void ProgressIndication::writeProgress() written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences). + // If approximate cores number is known, display it. + auto cores_number = getApproximateCoresNumber(); + std::string profiling_msg; + if (cores_number != 0 && print_hardware_utilization) + { + WriteBufferFromOwnString profiling_msg_builder; + // Calculated cores number may be not accurate + // so it's better to print min(threads, cores). + UInt64 threads_number = getUsedThreadsCount(); + profiling_msg_builder << " Running " << threads_number << " threads on " + << std::min(cores_number, threads_number) << " cores"; + + auto memory_usage = getMemoryUsage(); + if (memory_usage != 0) + profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; + else + profiling_msg_builder << "."; + profiling_msg = profiling_msg_builder.str(); + } + /// If the approximate number of rows to process is known, we can display a progress bar and percentage. if (progress.total_rows_to_read || progress.total_raw_bytes_to_read) { @@ -215,7 +236,7 @@ void ProgressIndication::writeProgress() if (show_progress_bar) { - ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%"); + ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%") - profiling_msg.length(); if (width_of_progress_bar > 0) { std::string bar @@ -231,23 +252,7 @@ void ProgressIndication::writeProgress() message << ' ' << (99 * current_count / max_count) << '%'; } - // If approximate cores number is known, display it. - auto cores_number = getApproximateCoresNumber(); - if (cores_number != 0) - { - // Calculated cores number may be not accurate - // so it's better to print min(threads, cores). - UInt64 threads_number = getUsedThreadsCount(); - message << " Running " << threads_number << " threads on " - << std::min(cores_number, threads_number) << " cores"; - - auto memory_usage = getMemoryUsage(); - if (memory_usage != 0) - message << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; - else - message << "."; - } - + message << profiling_msg; message << CLEAR_TO_END_OF_LINE; ++increment; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 3d9bbc7f3ff..9b1b2b0b145 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -60,6 +60,8 @@ public: void updateThreadEventData(HostToThreadTimesMap & new_thread_data); + bool print_hardware_utilization = false; + private: size_t getUsedThreadsCount() const; From 23602f46075821871fda78eaa238dd4a8eac9864 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 30 Aug 2021 23:29:09 +0800 Subject: [PATCH 14/89] Speed up part loading for JBOD --- src/Storages/MergeTree/MergeTreeData.cpp | 485 +++++++++++++++-------- src/Storages/MergeTree/MergeTreeData.h | 15 + 2 files changed, 333 insertions(+), 167 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8821991bae3..b88a152e1cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -897,6 +897,261 @@ Int64 MergeTreeData::getMaxBlockNumber() const return max_block_num; } +void MergeTreeData::loadDataPartsFromDisk( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + ThreadPool & pool, + size_t num_parts, + std::queue>> & parts_queue, + bool skip_sanity_checks, + const MergeTreeSettingsPtr & settings) +{ + /// Parallel loading of data parts. + pool.setMaxThreads(std::min(size_t(settings->max_part_loading_threads), num_parts)); + size_t num_threads = pool.getMaxThreads(); + std::vector parts_per_thread(num_threads, num_parts / num_threads); + for (size_t i = 0ul; i < num_parts % num_threads; ++i) + ++parts_per_thread[i]; + + /// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal + /// others' tasks when finish current disk part loading process. + std::vector>> threads_parts(num_threads); + std::set remaining_thread_parts; + std::queue threads_queue; + for (size_t i = 0; i < num_threads; ++i) + { + remaining_thread_parts.insert(i); + threads_queue.push(i); + } + + while (!parts_queue.empty()) + { + assert(!threads_queue.empty()); + size_t i = threads_queue.front(); + auto & need_parts = parts_per_thread[i]; + assert(need_parts > 0); + auto & thread_parts = threads_parts[i]; + auto & current_parts = parts_queue.front(); + assert(!current_parts.empty()); + auto parts_to_grab = std::min(need_parts, current_parts.size()); + + thread_parts.insert(thread_parts.end(), current_parts.end() - parts_to_grab, current_parts.end()); + current_parts.resize(current_parts.size() - parts_to_grab); + need_parts -= parts_to_grab; + + /// Before processing next thread, change disk if possible. + /// Different threads will likely start loading parts from different disk, + /// which may improve read parallelism for JBOD. + + /// If current disk still has some parts, push it to the tail. + if (!current_parts.empty()) + parts_queue.push(std::move(current_parts)); + parts_queue.pop(); + + /// If current thread still want some parts, push it to the tail. + if (need_parts > 0) + threads_queue.push(i); + threads_queue.pop(); + } + assert(threads_queue.empty()); + assert(std::all_of(threads_parts.begin(), threads_parts.end(), [](const std::vector> & parts) + { + return !parts.empty(); + })); + + size_t suspicious_broken_parts = 0; + size_t suspicious_broken_parts_bytes = 0; + std::atomic has_adaptive_parts = false; + std::atomic has_non_adaptive_parts = false; + + std::mutex mutex; + auto load_part = [&](const String & part_name, const DiskPtr & part_disk_ptr) + { + auto part_opt = MergeTreePartInfo::tryParsePartName(part_name, format_version); + if (!part_opt) + return; + const auto & part_info = *part_opt; + auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); + auto part = createPart(part_name, part_info, single_disk_volume, part_name); + bool broken = false; + + String part_path = fs::path(relative_data_path) / part_name; + String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + if (part_disk_ptr->exists(marker_path)) + { + /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist + size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); + LOG_WARNING(log, + "Detaching stale part {}{} (size: {}), which should have been deleted after a move. " + "That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", + getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); + std::lock_guard loading_lock(mutex); + broken_parts_to_detach.push_back(part); + ++suspicious_broken_parts; + suspicious_broken_parts_bytes += size_of_part; + return; + } + + try + { + part->loadColumnsChecksumsIndexes(require_part_metadata, true); + } + catch (const Exception & e) + { + /// Don't count the part as broken if there is not enough memory to load it. + /// In fact, there can be many similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + if (isNotEnoughMemoryErrorCode(e.code())) + throw; + + broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); + } + catch (...) + { + broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + /// Ignore broken parts that can appear as a result of hard server restart. + if (broken) + { + /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist + size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); + + LOG_ERROR(log, + "Detaching broken part {}{} (size: {}). " + "If it happened after update, it is likely because of backward incompability. " + "You need to resolve this manually", + getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); + std::lock_guard loading_lock(mutex); + broken_parts_to_detach.push_back(part); + ++suspicious_broken_parts; + suspicious_broken_parts_bytes += size_of_part; + return; + } + if (!part->index_granularity_info.is_adaptive) + has_non_adaptive_parts.store(true, std::memory_order_relaxed); + else + has_adaptive_parts.store(true, std::memory_order_relaxed); + + part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Committed); + + std::lock_guard loading_lock(mutex); + auto [it, inserted] = data_parts_indexes.insert(part); + /// Remove duplicate parts with the same checksum. + if (!inserted) + { + if ((*it)->checksums.getTotalChecksumHex() == part->checksums.getTotalChecksumHex()) + { + LOG_ERROR(log, "Remove duplicate part {}", part->getFullPath()); + duplicate_parts_to_remove.push_back(part); + } + else + throw Exception("Part " + part->name + " already exists but with different checksums", ErrorCodes::DUPLICATE_DATA_PART); + } + + addPartContributionToDataVolume(part); + }; + + std::mutex part_select_mutex; + try + { + for (size_t thread = 0; thread < num_threads; ++thread) + { + pool.scheduleOrThrowOnError([&, thread] + { + while (true) + { + std::pair thread_part; + { + const std::lock_guard lock{part_select_mutex}; + + if (remaining_thread_parts.empty()) + return; + + /// Steal task if nothing to do + auto thread_idx = thread; + if (threads_parts[thread].empty()) + { + // Try random steal tasks from the next thread + std::uniform_int_distribution distribution(0, remaining_thread_parts.size() - 1); + auto it = remaining_thread_parts.begin(); + std::advance(it, distribution(thread_local_rng)); + thread_idx = *it; + } + auto & thread_parts = threads_parts[thread_idx]; + thread_part = thread_parts.back(); + thread_parts.pop_back(); + if (thread_parts.empty()) + remaining_thread_parts.erase(thread_idx); + } + load_part(thread_part.first, thread_part.second); + } + }); + } + } + catch (...) + { + /// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad. + pool.wait(); + throw; + } + + pool.wait(); + + if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) + throw Exception( + "Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", + ErrorCodes::LOGICAL_ERROR); + + has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; + + if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) + throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, + "Suspiciously many ({}) broken parts to remove.", + suspicious_broken_parts); + + if (suspicious_broken_parts_bytes > settings->max_suspicious_broken_parts_bytes && !skip_sanity_checks) + throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, + "Suspiciously big size ({}) of all broken parts to remove.", + formatReadableSizeWithBinarySuffix(suspicious_broken_parts_bytes)); +} + + +void MergeTreeData::loadDataPartsFromWAL( + DataPartsVector & /* broken_parts_to_detach */, + DataPartsVector & duplicate_parts_to_remove, + MutableDataPartsVector & parts_from_wal, + DataPartsLock & part_lock) +{ + for (auto & part : parts_from_wal) + { + if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + continue; + + part->modification_time = time(nullptr); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Committed); + + auto [it, inserted] = data_parts_indexes.insert(part); + if (!inserted) + { + if ((*it)->checksums.getTotalChecksumHex() == part->checksums.getTotalChecksumHex()) + { + LOG_ERROR(log, "Remove duplicate part {}", part->getFullPath()); + duplicate_parts_to_remove.push_back(part); + } + else + throw Exception("Part " + part->name + " already exists but with different checksums", ErrorCodes::DUPLICATE_DATA_PART); + } + + addPartContributionToDataVolume(part); + } +} + void MergeTreeData::loadDataParts(bool skip_sanity_checks) { @@ -904,7 +1159,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto metadata_snapshot = getInMemoryMetadataPtr(); const auto settings = getSettings(); - std::vector> part_names_with_disks; MutableDataPartsVector parts_from_wal; Strings part_file_names; @@ -934,193 +1188,90 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - /// Reversed order to load part from low priority disks firstly. - /// Used for keep part on low priority disk if duplication found - for (auto disk_it = disks.rbegin(); disk_it != disks.rend(); ++disk_it) + /// Collect part names by disk. + std::map>> disk_part_map; + std::map disk_wal_part_map; + ThreadPool pool(disks.size()); + std::mutex wal_init_lock; + for (const auto & disk_ptr : disks) { - auto disk_ptr = *disk_it; + auto & disk_parts = disk_part_map[disk_ptr->getName()]; + auto & disk_wal_parts = disk_wal_part_map[disk_ptr->getName()]; - for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) + pool.scheduleOrThrowOnError([&, disk_ptr]() { - /// Skip temporary directories, file 'format_version.txt' and directory 'detached'. - if (startsWith(it->name(), "tmp") - || it->name() == MergeTreeData::FORMAT_VERSION_FILE_NAME - || it->name() == MergeTreeData::DETACHED_DIR_NAME) - continue; - - if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) - part_names_with_disks.emplace_back(it->name(), disk_ptr); - else if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) + for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) { - /// Create and correctly initialize global WAL object - write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); - for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext())) - parts_from_wal.push_back(std::move(part)); + /// Skip temporary directories, file 'format_version.txt' and directory 'detached'. + if (startsWith(it->name(), "tmp") || it->name() == MergeTreeData::FORMAT_VERSION_FILE_NAME + || it->name() == MergeTreeData::DETACHED_DIR_NAME) + continue; + + if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + disk_parts.emplace_back(std::make_pair(it->name(), disk_ptr)); + else if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) + { + std::unique_lock lock(wal_init_lock); + if (write_ahead_log != nullptr) + throw Exception( + "There are multiple WAL files appeared in current storage policy. You need to resolve this manually", + ErrorCodes::CORRUPTED_DATA); + + write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); + for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext())) + disk_wal_parts.push_back(std::move(part)); + } + else if (settings->in_memory_parts_enable_wal) + { + MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); + for (auto && part : wal.restore(metadata_snapshot, getContext())) + disk_wal_parts.push_back(std::move(part)); + } } - else if (settings->in_memory_parts_enable_wal) - { - MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); - for (auto && part : wal.restore(metadata_snapshot, getContext())) - parts_from_wal.push_back(std::move(part)); - } - } - } - - auto part_lock = lockParts(); - data_parts_indexes.clear(); - - if (part_names_with_disks.empty() && parts_from_wal.empty()) - { - LOG_DEBUG(log, "There are no data parts"); - return; - } - - /// Parallel loading of data parts. - size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_names_with_disks.size()); - - std::mutex mutex; - - DataPartsVector broken_parts_to_detach; - size_t suspicious_broken_parts = 0; - size_t suspicious_broken_parts_bytes = 0; - - std::atomic has_adaptive_parts = false; - std::atomic has_non_adaptive_parts = false; - - ThreadPool pool(num_threads); - - for (auto & part_names_with_disk : part_names_with_disks) - { - pool.scheduleOrThrowOnError([&] - { - const auto & [part_name, part_disk_ptr] = part_names_with_disk; - - auto part_opt = MergeTreePartInfo::tryParsePartName(part_name, format_version); - - if (!part_opt) - return; - - auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); - auto part = createPart(part_name, *part_opt, single_disk_volume, part_name); - bool broken = false; - - String part_path = fs::path(relative_data_path) / part_name; - String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; - - if (part_disk_ptr->exists(marker_path)) - { - /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist - size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); - LOG_WARNING(log, - "Detaching stale part {}{} (size: {}), which should have been deleted after a move. " - "That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", - getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); - std::lock_guard loading_lock(mutex); - - broken_parts_to_detach.push_back(part); - - ++suspicious_broken_parts; - suspicious_broken_parts_bytes += size_of_part; - - return; - } - - try - { - part->loadColumnsChecksumsIndexes(require_part_metadata, true); - } - catch (const Exception & e) - { - /// Don't count the part as broken if there is not enough memory to load it. - /// In fact, there can be many similar situations. - /// But it is OK, because there is a safety guard against deleting too many parts. - if (isNotEnoughMemoryErrorCode(e.code())) - throw; - - broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - catch (...) - { - broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - /// Ignore broken parts that can appear as a result of hard server restart. - if (broken) - { - /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist - size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); - - LOG_ERROR(log, - "Detaching broken part {}{} (size: {}). " - "If it happened after update, it is likely because of backward incompability. " - "You need to resolve this manually", - getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); - std::lock_guard loading_lock(mutex); - - broken_parts_to_detach.push_back(part); - - ++suspicious_broken_parts; - suspicious_broken_parts_bytes += size_of_part; - - return; - } - - if (!part->index_granularity_info.is_adaptive) - has_non_adaptive_parts.store(true, std::memory_order_relaxed); - else - has_adaptive_parts.store(true, std::memory_order_relaxed); - - part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); - - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); - - std::lock_guard loading_lock(mutex); - - if (!data_parts_indexes.insert(part).second) - throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Part {} already exists", part->name); - - addPartContributionToDataVolume(part); }); } pool.wait(); - for (auto & part : parts_from_wal) + for (auto & [_, disk_wal_parts] : disk_wal_part_map) + parts_from_wal.insert( + parts_from_wal.end(), std::make_move_iterator(disk_wal_parts.begin()), std::make_move_iterator(disk_wal_parts.end())); + + size_t num_parts = 0; + std::queue>> parts_queue; + for (auto & [_, disk_parts] : disk_part_map) { - if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + if (disk_parts.empty()) continue; - - part->modification_time = time(nullptr); - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); - - if (!data_parts_indexes.insert(part).second) - throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); - - addPartContributionToDataVolume(part); + num_parts += disk_parts.size(); + parts_queue.push(std::move(disk_parts)); } - if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) - throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR); + auto part_lock = lockParts(); + data_parts_indexes.clear(); - has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; + if (num_parts == 0 && parts_from_wal.empty()) + { + LOG_DEBUG(log, "There are no data parts"); + return; + } - if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) - throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, - "Suspiciously many ({}) broken parts to remove.", - suspicious_broken_parts); - if (suspicious_broken_parts_bytes > settings->max_suspicious_broken_parts_bytes && !skip_sanity_checks) - throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, - "Suspiciously big size ({}) of all broken parts to remove.", - formatReadableSizeWithBinarySuffix(suspicious_broken_parts_bytes)); + DataPartsVector broken_parts_to_detach; + DataPartsVector duplicate_parts_to_remove; + + if (num_parts > 0) + loadDataPartsFromDisk( + broken_parts_to_detach, duplicate_parts_to_remove, pool, num_parts, parts_queue, skip_sanity_checks, settings); + + if (!parts_from_wal.empty()) + loadDataPartsFromWAL(broken_parts_to_detach, duplicate_parts_to_remove, parts_from_wal, part_lock); for (auto & part : broken_parts_to_detach) part->renameToDetached("broken-on-start"); /// detached parts must not have '_' in prefixes + for (auto & part : duplicate_parts_to_remove) + part->remove(); /// Delete from the set of current parts those parts that are covered by another part (those parts that /// were merged), but that for some reason are still not deleted from the filesystem. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e7f1db8f3ec..b87a756bf9f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1143,6 +1143,21 @@ private: /// Returns default settings for storage with possible changes from global config. virtual std::unique_ptr getDefaultSettings() const = 0; + + void loadDataPartsFromDisk( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + ThreadPool & pool, + size_t num_parts, + std::queue>> & parts_queue, + bool skip_sanity_checks, + const MergeTreeSettingsPtr & settings); + + void loadDataPartsFromWAL( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + MutableDataPartsVector & parts_from_wal, + DataPartsLock & part_lock); }; /// RAII struct to record big parts that are submerging or emerging. From 50e2b064881e9da8d97edc12b71a0760e0135628 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 14:25:00 +0300 Subject: [PATCH 15/89] change branch for boringssl --- .gitmodules | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..a8b94cbfa6e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -213,6 +213,7 @@ [submodule "contrib/boringssl"] path = contrib/boringssl url = https://github.com/ClickHouse-Extras/boringssl.git + branch = MergeWithUpstream [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git From 7cef607cb92239f1544e0afbf0da2430fb5997cf Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 15:01:07 +0300 Subject: [PATCH 16/89] move on merge branch --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index 4c787e9d70c..486b2c0a869 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit 4c787e9d70c370d51baea714e7b73910be2a4c28 +Subproject commit 486b2c0a869fa8024c8a13e5bcefdd4a3d919947 From 562138c9fa14ec519c6ccd873d8787843eb11e6b Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Wed, 13 Oct 2021 15:01:21 +0300 Subject: [PATCH 17/89] codegen_fuzzer: removing errors on warnings from protobuf-generated code in more gentle way --- src/Parsers/fuzzers/CMakeLists.txt | 2 -- src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt index 2840dc72c0a..bb52101c847 100644 --- a/src/Parsers/fuzzers/CMakeLists.txt +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -7,7 +7,5 @@ target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZ add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) -string(REPLACE " -Werror" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") add_subdirectory(codegen_fuzzer) -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror") diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index f55bb3b3fb9..370396a5e8e 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -7,7 +7,9 @@ set(CMAKE_INCLUDE_CURRENT_DIR TRUE) add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) +set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") + target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") -target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) \ No newline at end of file +target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) From d2dfbb5ab627ebe8f607fcc835e4a5a18737a303 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 15:25:44 +0300 Subject: [PATCH 18/89] Remove trash from MergeTreeReadPool --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 24 ------------------- src/Storages/MergeTree/MergeTreeReadPool.h | 3 --- .../MergeTreeThreadSelectProcessor.cpp | 11 ++++----- .../00167_read_bytes_from_fs.reference | 2 ++ .../1_stateful/00167_read_bytes_from_fs.sql | 7 ++++++ 5 files changed, 13 insertions(+), 34 deletions(-) create mode 100644 tests/queries/1_stateful/00167_read_bytes_from_fs.reference create mode 100644 tests/queries/1_stateful/00167_read_bytes_from_fs.sql diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index d08cec24184..4bb247f1369 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -142,30 +142,6 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); } -MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const -{ - MarkRanges all_part_ranges; - - /// Inefficient in presence of large number of data parts. - for (const auto & part_ranges : parts_ranges) - { - if (part_ranges.data_part.get() == &part) - { - all_part_ranges = part_ranges.ranges; - break; - } - } - if (all_part_ranges.empty()) - throw Exception("Trying to read marks range [" + std::to_string(from.begin) + ", " + std::to_string(from.end) + "] from part '" - + part.getFullPath() + "' which has no ranges in this query", ErrorCodes::LOGICAL_ERROR); - - auto begin = std::lower_bound(all_part_ranges.begin(), all_part_ranges.end(), from, [] (const auto & f, const auto & s) { return f.begin < s.begin; }); - if (begin == all_part_ranges.end()) - begin = std::prev(all_part_ranges.end()); - begin->begin = from.begin; - return MarkRanges(begin, all_part_ranges.end()); -} - Block MergeTreeReadPool::getHeader() const { return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals(), data.getStorageID()); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 9949bdf86f8..380b132b806 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -85,9 +85,6 @@ public: */ void profileFeedback(const ReadBufferFromFileBase::ProfileInfo info); - /// This method tells which mark ranges we have to read if we start from @from mark range - MarkRanges getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const; - Block getHeader() const; private: diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 4eb6bc4b2e2..6a8ef860c87 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -68,18 +68,16 @@ bool MergeTreeThreadSelectProcessor::getNewTask() if (!reader) { - auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); - if (use_uncompressed_cache) owned_uncompressed_cache = storage.getContext()->getUncompressedCache(); owned_mark_cache = storage.getContext()->getMarkCache(); - reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); } @@ -88,14 +86,13 @@ bool MergeTreeThreadSelectProcessor::getNewTask() /// in other case we can reuse readers, anyway they will be "seeked" to required mark if (part_name != last_readed_part_name) { - auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); /// retain avg_value_size_hints - reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); } diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.reference b/tests/queries/1_stateful/00167_read_bytes_from_fs.reference new file mode 100644 index 00000000000..05b54da2ac7 --- /dev/null +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.reference @@ -0,0 +1,2 @@ +468426149779992039 +1 diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql new file mode 100644 index 00000000000..c3bdaea7abe --- /dev/null +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -0,0 +1,7 @@ +SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40 + +SYSTEM FLUSH LOGS; + +-- We had a bug which lead to additional compressed data read. hits compressed size if about 1.2G, but we read more then 3GB. +-- Small additional reads still possible, so we compare with about 1.5Gb. +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and type = 'QueryFinish' From 7e85b7e407838e55ee290f5747684bb6c95b44bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 15:27:42 +0300 Subject: [PATCH 19/89] Remove accident change --- docker/test/stateless/run.sh | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ebb72111e96..ed721690281 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,23 +45,6 @@ else sudo clickhouse start fi -echo " -set follow-fork-mode child -handle all noprint -handle SIGSEGV stop print -handle SIGBUS stop print -handle SIGABRT stop print -continue -thread apply all backtrace -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 006a9a7c7d9e9d645c4bdf9112eafb1ed076091f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Oct 2021 16:26:54 +0300 Subject: [PATCH 20/89] Print more info about memory utilization --- src/Common/ProgressIndication.cpp | 20 ++++++++++++-------- src/Common/ProgressIndication.h | 8 +++++++- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 4510952cc71..1f8fc949886 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -1,9 +1,11 @@ #include "ProgressIndication.h" +#include #include #include #include #include #include +#include "Common/formatReadable.h" #include #include #include "IO/WriteBufferFromString.h" @@ -114,16 +116,17 @@ UInt64 ProgressIndication::getApproximateCoresNumber() const }); } -UInt64 ProgressIndication::getMemoryUsage() const +ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const { - return std::accumulate(thread_data.cbegin(), thread_data.cend(), ZERO, - [](UInt64 acc, auto const & host_data) + return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{}, + [](MemoryUsage const & acc, auto const & host_data) { - return acc + std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, + auto host_usage = std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, [](UInt64 memory, auto const & data) { return memory + data.second.memory_usage; }); + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } @@ -202,11 +205,12 @@ void ProgressIndication::writeProgress() profiling_msg_builder << " Running " << threads_number << " threads on " << std::min(cores_number, threads_number) << " cores"; - auto memory_usage = getMemoryUsage(); + auto [memory_usage, max_host_usage] = getMemoryUsage(); if (memory_usage != 0) - profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; - else - profiling_msg_builder << "."; + profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used"; + if (thread_data.size() > 1 && max_host_usage) + profiling_msg_builder << " total (per host max: " << formatReadableSizeWithDecimalSuffix(max_host_usage) << ")"; + profiling_msg_builder << "."; profiling_msg = profiling_msg_builder.str(); } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 9b1b2b0b145..d31ed8df0ba 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -68,7 +68,13 @@ private: UInt64 getApproximateCoresNumber() const; - UInt64 getMemoryUsage() const; + struct MemoryUsage + { + UInt64 total = 0; + UInt64 max = 0; + }; + + MemoryUsage getMemoryUsage() const; /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. From 886d10c3ea19c04a35ac43e175e0730a1148adc9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:58:33 +0300 Subject: [PATCH 21/89] Start server under gdb in functional tests --- docker/test/stateless/run.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ce1d1b59a55..ec0af024b8b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,6 +45,23 @@ else sudo clickhouse start fi +echo " +set follow-fork-mode child +handle all noprint +handle SIGSEGV stop print +handle SIGBUS stop print +handle SIGABRT stop print +continue +thread apply all backtrace +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 2473bc5affadd1cb397ef810a262845fd8900220 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 18:12:04 +0300 Subject: [PATCH 22/89] Fix test --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index c3bdaea7abe..341730bd82d 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,7 +1,7 @@ -SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40 +SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; SYSTEM FLUSH LOGS; --- We had a bug which lead to additional compressed data read. hits compressed size if about 1.2G, but we read more then 3GB. +-- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and type = 'QueryFinish' +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and databaser=currentDatabase() and type = 'QueryFinish'; From 98b555e7f77e0c134775fd9abb344977645f29de Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 19:21:11 +0300 Subject: [PATCH 23/89] Update run.sh --- docker/test/stateless/run.sh | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ec0af024b8b..ce1d1b59a55 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,23 +45,6 @@ else sudo clickhouse start fi -echo " -set follow-fork-mode child -handle all noprint -handle SIGSEGV stop print -handle SIGBUS stop print -handle SIGABRT stop print -continue -thread apply all backtrace -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 91084895cee130a36ddb6e15c9727e6da6b607bf Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 20:18:51 +0300 Subject: [PATCH 24/89] attemp to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index 486b2c0a869..a139bb3cb95 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit 486b2c0a869fa8024c8a13e5bcefdd4a3d919947 +Subproject commit a139bb3cb9598c7d92dc69aa6962e3ea4fd18982 From 2957971ee30b9507cb73d6b6c90a1ebee87914fd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 21:22:02 +0300 Subject: [PATCH 25/89] Remove some last streams. --- src/Core/ExternalTable.cpp | 1 - src/DataStreams/BlockIO.cpp | 1 - .../MaterializingBlockOutputStream.h | 34 --- src/DataStreams/OneBlockInputStream.h | 41 ---- src/DataStreams/SquashingBlockInputStream.cpp | 32 --- src/DataStreams/SquashingBlockInputStream.h | 31 --- .../SquashingBlockOutputStream.cpp | 54 ----- src/DataStreams/SquashingBlockOutputStream.h | 39 ---- .../gtest_blocks_size_merging_streams.cpp | 32 +-- src/Databases/DatabaseReplicated.h | 1 - .../MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Dictionaries/LibraryDictionarySource.cpp | 1 - src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Formats/FormatFactory.cpp | 1 - src/Interpreters/Context.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 2 - src/Interpreters/InterpreterSelectQuery.cpp | 51 ++--- src/Interpreters/InterpreterSelectQuery.h | 4 +- src/Interpreters/MergeJoin.cpp | 7 +- src/Interpreters/MutationsInterpreter.cpp | 1 - .../PipelineExecutingBlockInputStream.cpp | 124 ----------- .../PipelineExecutingBlockInputStream.h | 44 ---- .../Formats/InputStreamFromInputFormat.h | 67 ------ src/Processors/Pipe.cpp | 10 +- src/Processors/QueryPipeline.cpp | 10 +- src/Processors/QueryPipelineBuilder.cpp | 10 +- src/Processors/QueryPlan/ExpressionStep.cpp | 1 - .../Sources/SourceFromInputStream.cpp | 195 ------------------ .../Sources/SourceFromInputStream.h | 77 ------- .../Transforms/AggregatingTransform.h | 3 - src/Storages/HDFS/StorageHDFS.cpp | 16 +- src/Storages/Kafka/KafkaSource.cpp | 2 - src/Storages/Kafka/StorageKafka.cpp | 1 - src/Storages/MergeTree/MergeTask.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 - .../MergeTree/MergeTreeDataWriter.cpp | 2 - src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 1 - .../MaterializedPostgreSQLConsumer.h | 1 - .../StorageMaterializedPostgreSQL.cpp | 1 - src/Storages/RabbitMQ/RabbitMQSource.cpp | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 - .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDictionary.cpp | 1 - src/Storages/StorageDistributed.cpp | 1 - src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageInput.cpp | 1 - src/Storages/StorageMaterializedView.cpp | 1 - src/Storages/StorageMerge.cpp | 4 +- src/Storages/StorageMongoDB.cpp | 1 - src/Storages/StorageMySQL.cpp | 1 - src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 2 - src/Storages/StorageSQLite.cpp | 1 - src/Storages/StorageURL.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 1 - 58 files changed, 67 insertions(+), 874 deletions(-) delete mode 100644 src/DataStreams/MaterializingBlockOutputStream.h delete mode 100644 src/DataStreams/OneBlockInputStream.h delete mode 100644 src/DataStreams/SquashingBlockInputStream.cpp delete mode 100644 src/DataStreams/SquashingBlockInputStream.h delete mode 100644 src/DataStreams/SquashingBlockOutputStream.cpp delete mode 100644 src/DataStreams/SquashingBlockOutputStream.h delete mode 100644 src/Processors/Executors/PipelineExecutingBlockInputStream.cpp delete mode 100644 src/Processors/Executors/PipelineExecutingBlockInputStream.h delete mode 100644 src/Processors/Formats/InputStreamFromInputFormat.h delete mode 100644 src/Processors/Sources/SourceFromInputStream.cpp delete mode 100644 src/Processors/Sources/SourceFromInputStream.h diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 55ad748868e..7619d60d84e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -12,7 +12,6 @@ #include #include -#include #include #include #include diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 5f1abdaf806..692b69388ea 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB { diff --git a/src/DataStreams/MaterializingBlockOutputStream.h b/src/DataStreams/MaterializingBlockOutputStream.h deleted file mode 100644 index 64c2bc12a57..00000000000 --- a/src/DataStreams/MaterializingBlockOutputStream.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Converts columns-constants to full columns ("materializes" them). - */ -class MaterializingBlockOutputStream : public IBlockOutputStream -{ -public: - MaterializingBlockOutputStream(const BlockOutputStreamPtr & output_, const Block & header_) - : output{output_}, header(header_) {} - - Block getHeader() const override { return header; } - void write(const Block & block) override { output->write(materializeBlock(block)); } - void flush() override { output->flush(); } - void writePrefix() override { output->writePrefix(); } - void writeSuffix() override { output->writeSuffix(); } - void setRowsBeforeLimit(size_t rows_before_limit) override { output->setRowsBeforeLimit(rows_before_limit); } - void setTotals(const Block & totals) override { output->setTotals(materializeBlock(totals)); } - void setExtremes(const Block & extremes) override { output->setExtremes(materializeBlock(extremes)); } - void onProgress(const Progress & progress) override { output->onProgress(progress); } - String getContentType() const override { return output->getContentType(); } - -private: - BlockOutputStreamPtr output; - Block header; -}; - -} diff --git a/src/DataStreams/OneBlockInputStream.h b/src/DataStreams/OneBlockInputStream.h deleted file mode 100644 index d401082ce62..00000000000 --- a/src/DataStreams/OneBlockInputStream.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** A stream of blocks from which you can read one block. - */ -class OneBlockInputStream : public IBlockInputStream -{ -public: - explicit OneBlockInputStream(Block block_) : block(std::move(block_)) { block.checkNumberOfRows(); } - - String getName() const override { return "One"; } - - Block getHeader() const override - { - Block res; - for (const auto & elem : block) - res.insert({ elem.column->cloneEmpty(), elem.type, elem.name }); - return res; - } - -protected: - Block readImpl() override - { - if (has_been_read) - return Block(); - - has_been_read = true; - return block; - } - -private: - Block block; - bool has_been_read = false; -}; - -} diff --git a/src/DataStreams/SquashingBlockInputStream.cpp b/src/DataStreams/SquashingBlockInputStream.cpp deleted file mode 100644 index e13dee37008..00000000000 --- a/src/DataStreams/SquashingBlockInputStream.cpp +++ /dev/null @@ -1,32 +0,0 @@ -#include - - -namespace DB -{ - -SquashingBlockInputStream::SquashingBlockInputStream( - const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory) - : header(src->getHeader()), transform(min_block_size_rows, min_block_size_bytes, reserve_memory) -{ - children.emplace_back(src); -} - - -Block SquashingBlockInputStream::readImpl() -{ - while (!all_read) - { - Block block = children[0]->read(); - if (!block) - all_read = true; - - auto squashed_block = transform.add(std::move(block)); - if (squashed_block) - { - return squashed_block; - } - } - return {}; -} - -} diff --git a/src/DataStreams/SquashingBlockInputStream.h b/src/DataStreams/SquashingBlockInputStream.h deleted file mode 100644 index c2732d520cc..00000000000 --- a/src/DataStreams/SquashingBlockInputStream.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Merging consecutive blocks of stream to specified minimum size. - */ -class SquashingBlockInputStream : public IBlockInputStream -{ -public: - SquashingBlockInputStream(const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes, - bool reserve_memory = false); - - String getName() const override { return "Squashing"; } - - Block getHeader() const override { return header; } - -protected: - Block readImpl() override; - -private: - Block header; - SquashingTransform transform; - bool all_read = false; -}; - -} diff --git a/src/DataStreams/SquashingBlockOutputStream.cpp b/src/DataStreams/SquashingBlockOutputStream.cpp deleted file mode 100644 index ab12f66590f..00000000000 --- a/src/DataStreams/SquashingBlockOutputStream.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include - - -namespace DB -{ - -SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr dst, Block header_, size_t min_block_size_rows, size_t min_block_size_bytes) - : output(std::move(dst)), header(std::move(header_)), transform(min_block_size_rows, min_block_size_bytes) -{ -} - - -void SquashingBlockOutputStream::write(const Block & block) -{ - auto squashed_block = transform.add(block); - if (squashed_block) - output->write(squashed_block); -} - - -void SquashingBlockOutputStream::finalize() -{ - if (all_written) - return; - - all_written = true; - - auto squashed_block = transform.add({}); - if (squashed_block) - output->write(squashed_block); -} - - -void SquashingBlockOutputStream::flush() -{ - if (!disable_flush) - finalize(); - output->flush(); -} - - -void SquashingBlockOutputStream::writePrefix() -{ - output->writePrefix(); -} - - -void SquashingBlockOutputStream::writeSuffix() -{ - finalize(); - output->writeSuffix(); -} - -} diff --git a/src/DataStreams/SquashingBlockOutputStream.h b/src/DataStreams/SquashingBlockOutputStream.h deleted file mode 100644 index 7828ad7e96d..00000000000 --- a/src/DataStreams/SquashingBlockOutputStream.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Merging consecutive blocks of stream to specified minimum size. - */ -class SquashingBlockOutputStream : public IBlockOutputStream -{ -public: - SquashingBlockOutputStream(BlockOutputStreamPtr dst, Block header_, size_t min_block_size_rows, size_t min_block_size_bytes); - - Block getHeader() const override { return header; } - void write(const Block & block) override; - - void flush() override; - void writePrefix() override; - void writeSuffix() override; - - /// Don't write blocks less than specified size even when flush method was called by user. - void disableFlush() { disable_flush = true; } - -private: - BlockOutputStreamPtr output; - Block header; - - SquashingTransform transform; - bool all_written = false; - - void finalize(); - - bool disable_flush = false; -}; - -} diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index 71cf41fcbab..7625fe8fa09 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include using namespace DB; @@ -88,15 +88,18 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) pipe.addTransform(std::move(transform)); QueryPipeline pipeline(std::move(pipe)); - pipeline.setNumThreads(1); - auto stream = std::make_shared(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); size_t total_rows = 0; - auto block1 = stream->read(); - auto block2 = stream->read(); - auto block3 = stream->read(); + Block block1; + Block block2; + Block block3; + executor.pull(block1); + executor.pull(block2); + executor.pull(block3); - EXPECT_EQ(stream->read(), Block()); + Block tmp_block; + ASSERT_FALSE(executor.pull(tmp_block)); for (const auto & block : {block1, block2, block3}) total_rows += block.rows(); @@ -132,14 +135,17 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) pipe.addTransform(std::move(transform)); QueryPipeline pipeline(std::move(pipe)); - pipeline.setNumThreads(1); - auto stream = std::make_shared(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); - auto block1 = stream->read(); - auto block2 = stream->read(); - auto block3 = stream->read(); + Block block1; + Block block2; + Block block3; + executor.pull(block1); + executor.pull(block2); + executor.pull(block3); - EXPECT_EQ(stream->read(), Block()); + Block tmp_block; + ASSERT_FALSE(executor.pull(tmp_block)); EXPECT_EQ(block1.rows(), (1000 + 1500 + 1400) / 3); EXPECT_EQ(block2.rows(), (1000 + 1500 + 1400) / 3); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 60526a1e5b0..aecbc1474f8 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -5,7 +5,6 @@ #include #include #include -#include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 5a714645978..f4a5b6b5e4e 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -14,7 +14,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 551bb1ee2dd..f117cfb179e 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -1,6 +1,5 @@ #include "LibraryDictionarySource.h" -#include #include #include #include diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 80081e67b42..b0be90c26a5 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +17,7 @@ #include "registerDictionaries.h" #include #include +#include namespace DB diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d3ff5cbf8a7..ed9f9d52b94 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6e729eb2370..46fa31e2738 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index a36941ea07a..8a31917caef 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -20,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85cc889319f..b4ffa15a869 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -64,7 +63,6 @@ #include #include #include -#include #include #include #include @@ -158,24 +156,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( ContextPtr context_, const SelectQueryOptions & options_, const Names & required_result_column_names_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, nullptr, options_, required_result_column_names_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) { } -InterpreterSelectQuery::InterpreterSelectQuery( - const ASTPtr & query_ptr_, - ContextPtr context_, - const BlockInputStreamPtr & input_, - const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, input_, std::nullopt, nullptr, options_.copy().noSubquery()) -{} - InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, Pipe input_pipe_, const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) + : InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) {} InterpreterSelectQuery::InterpreterSelectQuery( @@ -184,7 +174,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) {} InterpreterSelectQuery::~InterpreterSelectQuery() = default; @@ -268,7 +258,6 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, - const BlockInputStreamPtr & input_, std::optional input_pipe_, const StoragePtr & storage_, const SelectQueryOptions & options_, @@ -277,7 +266,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// NOTE: the query almost always should be cloned because it will be modified during analysis. : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) - , input(input_) , input_pipe(std::move(input_pipe_)) , log(&Poco::Logger::get("InterpreterSelectQuery")) , metadata_snapshot(metadata_snapshot_) @@ -294,13 +282,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(), ErrorCodes::TOO_DEEP_SUBQUERIES); - bool has_input = input || input_pipe; - if (input) - { - /// Read from prepared input. - source_header = input->getHeader(); - } - else if (input_pipe) + bool has_input = input_pipe != std::nullopt; + if (input_pipe) { /// Read from prepared input. source_header = input_pipe->getHeader(); @@ -450,17 +433,17 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!options.only_analyze) { - if (query.sampleSize() && (input || input_pipe || !storage || !storage->supportsSampling())) + if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling())) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); - if (query.final() && (input || input_pipe || !storage || !storage->supportsFinal())) + if (query.final() && (input_pipe || !storage || !storage->supportsFinal())) throw Exception( - (!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", + (!input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL); - if (query.prewhere() && (input || input_pipe || !storage || !storage->supportsPrewhere())) + if (query.prewhere() && (input_pipe || !storage || !storage->supportsPrewhere())) throw Exception( - (!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", + (!input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE); /// Save the new temporary tables in the query context @@ -578,7 +561,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { - executeImpl(query_plan, input, std::move(input_pipe)); + executeImpl(query_plan, std::move(input_pipe)); /// We must guarantee that result structure is the same as in getSampleBlock() /// @@ -926,7 +909,7 @@ static bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query) } -void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe) +void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional prepared_pipe) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -1010,13 +993,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } else { - if (prepared_input) - { - auto prepared_source_step - = std::make_unique(Pipe(std::make_shared(prepared_input)), context); - query_plan.addStep(std::move(prepared_source_step)); - } - else if (prepared_pipe) + if (prepared_pipe) { auto prepared_source_step = std::make_unique(std::move(*prepared_pipe), context); query_plan.addStep(std::move(prepared_source_step)); @@ -1580,7 +1557,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() { if (!expressions.prewhere_info) { - const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); + const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 99c95a8d624..21e15bc74bb 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -108,7 +108,6 @@ private: InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, - const BlockInputStreamPtr & input_, std::optional input_pipe, const StoragePtr & storage_, const SelectQueryOptions &, @@ -122,7 +121,7 @@ private: Block getSampleBlockImpl(); - void executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe); + void executeImpl(QueryPlan & query_plan, std::optional prepared_pipe); /// Different stages of query execution. @@ -198,7 +197,6 @@ private: TableLockHolder table_lock; /// Used when we read from prepared input, not table or subquery. - BlockInputStreamPtr input; std::optional input_pipe; Poco::Logger * log; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 9cfc43b92c5..1fc551334e2 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB @@ -592,9 +592,10 @@ void MergeJoin::mergeInMemoryRightBlocks() builder.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, 0, nullptr, 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); - auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); - while (Block block = sorted_input.read()) + Block block; + while (executor.pull(block)) { if (!block.rows()) continue; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2c12c4a6879..3d0813579ce 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp deleted file mode 100644 index bdfbbc2874e..00000000000 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp +++ /dev/null @@ -1,124 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipeline pipeline_) - : pipeline(std::make_unique(std::move(pipeline_))) -{ -} - -PipelineExecutingBlockInputStream::~PipelineExecutingBlockInputStream() = default; - -Block PipelineExecutingBlockInputStream::getHeader() const -{ - if (executor) - return executor->getHeader(); - - if (async_executor) - return async_executor->getHeader(); - - return pipeline->getHeader(); -} - -void PipelineExecutingBlockInputStream::createExecutor() -{ - if (pipeline->getNumThreads() > 1) - async_executor = std::make_unique(*pipeline); - else - executor = std::make_unique(*pipeline); - - is_execution_started = true; -} - -void PipelineExecutingBlockInputStream::readPrefixImpl() -{ - createExecutor(); -} - -Block PipelineExecutingBlockInputStream::readImpl() -{ - if (!is_execution_started) - createExecutor(); - - Block block; - bool can_continue = true; - while (can_continue) - { - if (executor) - can_continue = executor->pull(block); - else - can_continue = async_executor->pull(block); - - if (block) - return block; - } - - totals = executor ? executor->getTotalsBlock() - : async_executor->getTotalsBlock(); - - extremes = executor ? executor->getExtremesBlock() - : async_executor->getExtremesBlock(); - - return {}; -} - -inline static void throwIfExecutionStarted(bool is_execution_started, const char * method) -{ - if (is_execution_started) - throw Exception(String("Cannot call ") + method + - " for PipelineExecutingBlockInputStream because execution was started", - ErrorCodes::LOGICAL_ERROR); -} - -void PipelineExecutingBlockInputStream::cancel(bool kill) -{ - IBlockInputStream::cancel(kill); - - if (is_execution_started) - { - executor ? executor->cancel() - : async_executor->cancel(); - } -} - -void PipelineExecutingBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - throwIfExecutionStarted(is_execution_started, "setProgressCallback"); - pipeline->setProgressCallback(callback); -} - -void PipelineExecutingBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - throwIfExecutionStarted(is_execution_started, "setProcessListElement"); - IBlockInputStream::setProcessListElement(elem); - pipeline->setProcessListElement(elem); -} - -void PipelineExecutingBlockInputStream::setLimits(const StreamLocalLimits & limits_) -{ - throwIfExecutionStarted(is_execution_started, "setLimits"); - - if (limits_.mode == LimitsMode::LIMITS_TOTAL) - throw Exception("Total limits are not supported by PipelineExecutingBlockInputStream", - ErrorCodes::LOGICAL_ERROR); - - /// Local limits may be checked by IBlockInputStream itself. - IBlockInputStream::setLimits(limits_); -} - -void PipelineExecutingBlockInputStream::setQuota(const std::shared_ptr &) -{ - throw Exception("Quota is not supported by PipelineExecutingBlockInputStream", - ErrorCodes::LOGICAL_ERROR); -} - -} diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h deleted file mode 100644 index 68497938ad4..00000000000 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class QueryPipeline; -class PullingAsyncPipelineExecutor; -class PullingPipelineExecutor; - -/// Implement IBlockInputStream from QueryPipeline. -/// It's a temporary wrapper. -class PipelineExecutingBlockInputStream : public IBlockInputStream -{ -public: - explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_); - ~PipelineExecutingBlockInputStream() override; - - String getName() const override { return "PipelineExecuting"; } - Block getHeader() const override; - - void cancel(bool kill) override; - - /// Implement IBlockInputStream methods via QueryPipeline. - void setProgressCallback(const ProgressCallback & callback) final; - void setProcessListElement(QueryStatus * elem) final; - void setLimits(const StreamLocalLimits & limits_) final; - void setQuota(const std::shared_ptr & quota_) final; - -protected: - void readPrefixImpl() override; - Block readImpl() override; - -private: - std::unique_ptr pipeline; - /// One of executors is used. - std::unique_ptr executor; /// for single thread. - std::unique_ptr async_executor; /// for many threads. - bool is_execution_started = false; - - void createExecutor(); -}; - -} diff --git a/src/Processors/Formats/InputStreamFromInputFormat.h b/src/Processors/Formats/InputStreamFromInputFormat.h deleted file mode 100644 index 339f559ac9b..00000000000 --- a/src/Processors/Formats/InputStreamFromInputFormat.h +++ /dev/null @@ -1,67 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class InputStreamFromInputFormat : public IBlockInputStream -{ -public: - explicit InputStreamFromInputFormat(InputFormatPtr input_format_) - : input_format(std::move(input_format_)) - , port(input_format->getPort().getHeader(), input_format.get()) - { - connect(input_format->getPort(), port); - port.setNeeded(); - } - - String getName() const override { return input_format->getName(); } - Block getHeader() const override { return input_format->getPort().getHeader(); } - - void cancel(bool kill) override - { - input_format->cancel(); - IBlockInputStream::cancel(kill); - } - - const BlockMissingValues & getMissingValues() const override { return input_format->getMissingValues(); } - -protected: - - Block readImpl() override - { - while (true) - { - auto status = input_format->prepare(); - - switch (status) - { - case IProcessor::Status::Ready: - input_format->work(); - break; - - case IProcessor::Status::Finished: - return {}; - - case IProcessor::Status::PortFull: - return input_format->getPort().getHeader().cloneWithColumns(port.pull().detachColumns()); - - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); - } - } - } - -private: - InputFormatPtr input_format; - InputPort port; -}; - -} diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index ec288484ca3..acf46b95346 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -9,6 +8,7 @@ #include #include #include +#include #include #include @@ -164,13 +164,7 @@ Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, Output Pipe::Pipe(ProcessorPtr source) { - if (auto * source_from_input_stream = typeid_cast(source.get())) - { - /// Special case for SourceFromInputStream. Will remove it later. - totals_port = source_from_input_stream->getTotalsPort(); - extremes_port = source_from_input_stream->getExtremesPort(); - } - else if (source->getOutputs().size() != 1) + if (source->getOutputs().size() != 1) checkSource(*source); if (collected_processors) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index a76f99c9e00..98ac81f7217 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -121,7 +120,6 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. std::vector limits; - std::vector sources; std::vector remote_sources; std::unordered_set visited; @@ -151,9 +149,6 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) limits.emplace_back(limit); } - if (auto * source = typeid_cast(processor)) - sources.emplace_back(source); - if (auto * source = typeid_cast(processor)) remote_sources.emplace_back(source); } @@ -186,16 +181,13 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) } } - if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty())) + if (!rows_before_limit_at_least && (!limits.empty() || !remote_sources.empty())) { rows_before_limit_at_least = std::make_shared(); for (auto & limit : limits) limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : sources) - source->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : remote_sources) source->setRowsBeforeLimitCounter(rows_before_limit_at_least); } diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index 8ed413166da..08a568b48f1 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -468,7 +467,6 @@ void QueryPipelineBuilder::initRowsBeforeLimit() /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. std::vector limits; - std::vector sources; std::vector remote_sources; std::unordered_set visited; @@ -498,9 +496,6 @@ void QueryPipelineBuilder::initRowsBeforeLimit() limits.emplace_back(limit); } - if (auto * source = typeid_cast(processor)) - sources.emplace_back(source); - if (auto * source = typeid_cast(processor)) remote_sources.emplace_back(source); } @@ -533,16 +528,13 @@ void QueryPipelineBuilder::initRowsBeforeLimit() } } - if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty())) + if (!rows_before_limit_at_least && (!limits.empty() || !remote_sources.empty())) { rows_before_limit_at_least = std::make_shared(); for (auto & limit : limits) limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : sources) - source->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : remote_sources) source->setRowsBeforeLimitCounter(rows_before_limit_at_least); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index b4ff1a1281c..d1b9c2cad63 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Sources/SourceFromInputStream.cpp b/src/Processors/Sources/SourceFromInputStream.cpp deleted file mode 100644 index 57e449370a5..00000000000 --- a/src/Processors/Sources/SourceFromInputStream.cpp +++ /dev/null @@ -1,195 +0,0 @@ -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -SourceFromInputStream::SourceFromInputStream(BlockInputStreamPtr stream_, bool force_add_aggregating_info_) - : ISourceWithProgress(stream_->getHeader()) - , force_add_aggregating_info(force_add_aggregating_info_) - , stream(std::move(stream_)) -{ - init(); -} - -void SourceFromInputStream::init() -{ - const auto & sample = getPort().getHeader(); - for (auto & type : sample.getDataTypes()) - if (typeid_cast(type.get())) - has_aggregate_functions = true; -} - -void SourceFromInputStream::addTotalsPort() -{ - if (totals_port) - throw Exception("Totals port was already added for SourceFromInputStream.", ErrorCodes::LOGICAL_ERROR); - - outputs.emplace_back(outputs.front().getHeader(), this); - totals_port = &outputs.back(); -} - -void SourceFromInputStream::addExtremesPort() -{ - if (extremes_port) - throw Exception("Extremes port was already added for SourceFromInputStream.", ErrorCodes::LOGICAL_ERROR); - - outputs.emplace_back(outputs.front().getHeader(), this); - extremes_port = &outputs.back(); -} - -IProcessor::Status SourceFromInputStream::prepare() -{ - auto status = ISource::prepare(); - - if (status == Status::Finished) - { - is_generating_finished = true; - - /// Read postfix and get totals if needed. - if (!is_stream_finished && !isCancelled()) - return Status::Ready; - - if (totals_port && !totals_port->isFinished()) - { - if (has_totals) - { - if (!totals_port->canPush()) - return Status::PortFull; - - totals_port->push(std::move(totals)); - has_totals = false; - } - - totals_port->finish(); - } - - if (extremes_port && !extremes_port->isFinished()) - { - if (has_extremes) - { - if (!extremes_port->canPush()) - return Status::PortFull; - - extremes_port->push(std::move(extremes)); - has_extremes = false; - } - - extremes_port->finish(); - } - } - - return status; -} - -void SourceFromInputStream::work() -{ - if (!is_generating_finished) - { - try - { - ISource::work(); - } - catch (...) - { - /// Won't read suffix in case of exception. - is_stream_finished = true; - throw; - } - - return; - } - - if (is_stream_finished) - return; - - if (rows_before_limit) - { - const auto & info = stream->getProfileInfo(); - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - } - - stream->readSuffix(); - - if (auto totals_block = stream->getTotals()) - { - totals.setColumns(totals_block.getColumns(), 1); - has_totals = true; - } - - is_stream_finished = true; -} - -Chunk SourceFromInputStream::generate() -{ - if (is_stream_finished) - return {}; - - if (!is_stream_started) - { - stream->readPrefix(); - is_stream_started = true; - } - - auto block = stream->read(); - if (!block && !isCancelled()) - { - if (rows_before_limit) - { - const auto & info = stream->getProfileInfo(); - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - } - - stream->readSuffix(); - - if (auto totals_block = stream->getTotals()) - { - if (totals_block.rows() > 0) /// Sometimes we can get empty totals. Skip it. - { - totals.setColumns(totals_block.getColumns(), totals_block.rows()); - has_totals = true; - } - } - - if (auto extremes_block = stream->getExtremes()) - { - if (extremes_block.rows() > 0) /// Sometimes we can get empty extremes. Skip it. - { - extremes.setColumns(extremes_block.getColumns(), extremes_block.rows()); - has_extremes = true; - } - } - - is_stream_finished = true; - return {}; - } - - if (isCancelled()) - return {}; - -#ifndef NDEBUG - assertBlocksHaveEqualStructure(getPort().getHeader(), block, "SourceFromInputStream"); -#endif - - UInt64 num_rows = block.rows(); - Chunk chunk(block.getColumns(), num_rows); - - if (force_add_aggregating_info || has_aggregate_functions) - { - auto info = std::make_shared(); - info->bucket_num = block.info.bucket_num; - info->is_overflows = block.info.is_overflows; - chunk.setChunkInfo(std::move(info)); - } - - return chunk; -} - -} diff --git a/src/Processors/Sources/SourceFromInputStream.h b/src/Processors/Sources/SourceFromInputStream.h deleted file mode 100644 index 9649385909c..00000000000 --- a/src/Processors/Sources/SourceFromInputStream.h +++ /dev/null @@ -1,77 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class IBlockInputStream; -using BlockInputStreamPtr = std::shared_ptr; - -/// Wrapper for IBlockInputStream which implements ISourceWithProgress. -class SourceFromInputStream : public ISourceWithProgress -{ -public: - /// If force_add_aggregating_info is enabled, AggregatedChunkInfo (with bucket number and is_overflows flag) will be added to result chunk. - explicit SourceFromInputStream(BlockInputStreamPtr stream_, bool force_add_aggregating_info_ = false); - String getName() const override { return "SourceFromInputStream"; } - - Status prepare() override; - void work() override; - - Chunk generate() override; - - BlockInputStreamPtr & getStream() { return stream; } - - void addTotalsPort(); - void addExtremesPort(); - - OutputPort * getTotalsPort() const { return totals_port; } - OutputPort * getExtremesPort() const { return extremes_port; } - - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); } - - /// Implementation for methods from ISourceWithProgress. - void setLimits(const StreamLocalLimits & limits_) final { stream->setLimits(limits_); } - void setLeafLimits(const SizeLimits &) final { } - void setQuota(const std::shared_ptr & quota_) final { stream->setQuota(quota_); } - void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); } - void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); } - void addTotalRowsApprox(size_t value) final { stream->addTotalRowsApprox(value); } - - /// Stop reading from stream if output port is finished. - void onUpdatePorts() override - { - if (getPort().isFinished()) - cancel(); - } - -protected: - void onCancel() override { stream->cancel(false); } - -private: - bool has_aggregate_functions = false; - bool force_add_aggregating_info = false; - BlockInputStreamPtr stream; - - RowsBeforeLimitCounterPtr rows_before_limit; - - Chunk totals; - OutputPort * totals_port = nullptr; - bool has_totals = false; - - Chunk extremes; - OutputPort * extremes_port = nullptr; - bool has_extremes = false; - - bool is_generating_finished = false; - bool is_stream_finished = false; - bool is_stream_started = false; - - void init(); -}; - -} diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 1639bc4df4b..2a515fdf3be 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -24,9 +24,6 @@ public: Int32 bucket_num = -1; }; -class IBlockInputStream; -using BlockInputStreamPtr = std::shared_ptr; - using AggregatorList = std::list; using AggregatorListPtr = std::shared_ptr; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19385e526a7..12558054d81 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -23,6 +22,9 @@ #include #include #include +#include +#include +#include #include #include @@ -124,12 +126,13 @@ public: auto compression = chooseCompressionMethod(path, compression_method); read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri, path, getContext()->getGlobalContext()->getConfigRef()), compression); auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size); + pipeline = QueryPipeline(std::move(input_format)); - reader = std::make_shared(input_format); - reader->readPrefix(); + reader = std::make_unique(pipeline); } - if (auto res = reader->read()) + Block res; + if (reader->pull(res)) { Columns columns = res.getColumns(); UInt64 num_rows = res.rows(); @@ -153,15 +156,16 @@ public: return Chunk(std::move(columns), num_rows); } - reader->readSuffix(); reader.reset(); + pipeline.reset(); read_buf.reset(); } } private: std::unique_ptr read_buf; - BlockInputStreamPtr reader; + QueryPipeline pipeline; + std::unique_ptr reader; SourcesInfoPtr source_info; String uri; String format; diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 68fc17a97e5..30c5ef3e6cd 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -1,9 +1,7 @@ #include -#include #include #include -#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 903ea81946d..690c9cbd4d0 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 357659b3bbb..e17a3dcf544 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -19,7 +19,6 @@ #include "Processors/Merges/GraphiteRollupSortedTransform.h" #include "Processors/Merges/AggregatingSortedTransform.h" #include "Processors/Merges/VersionedCollapsingTransform.h" -#include "Processors/Executors/PipelineExecutingBlockInputStream.h" #include "DataStreams/TTLBlockInputStream.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 683689a6fa3..f4adee8c259 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include @@ -56,6 +55,7 @@ #include #include #include +#include #include #include @@ -3481,9 +3481,12 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc buf, metadata_snapshot->getPartitionKey().sample_block, local_context->getSettingsRef().max_block_size); - auto input_stream = std::make_shared(input_format); + QueryPipeline pipeline(std::move(input_format)); + PullingPipelineExecutor executor(pipeline); + + Block block; + executor.pull(block); - auto block = input_stream->read(); if (!block || !block.rows()) throw Exception( "Could not parse partition value: `" + partition_ast.fields_str + "`", diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 903f4cd27fc..1ee8423b0d1 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -28,7 +28,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d939312c0bb..50ab26ec470 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,8 +13,6 @@ #include #include #include -#include -#include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b8941fc9d84..6ad29d01ca6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,9 +4,9 @@ #include #include #include +#include #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index fdc30919ee7..9e138e9882a 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 5a99be7f38c..ff47866d587 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -7,7 +7,6 @@ #include #include #include -#include #include diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 0cd758cf49d..4e9e2c450b1 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 34edd06d3e2..b954ad3ab23 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0944a8f12d5..2b4f5e4a276 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 1ab168f772f..3a37e57e008 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -17,8 +17,8 @@ #include #include -#include #include +#include #include #include diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9747ea2dbf6..b774622d569 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -24,8 +24,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index c8bc215dd6c..e305d4c6183 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b9c15e19c33..df92b270542 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -59,7 +59,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index ec0bd5e5840..9aa5689aa66 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,8 +34,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index d707d7a6cdf..1138794adb0 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d2493ff7c43..5a9e8fc2461 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ea42b48cace..6492c9e07c0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -383,7 +383,7 @@ Pipe StorageMerge::createSources( { pipe = QueryPipelineBuilder::getPipe(InterpreterSelectQuery( modified_query_info.query, modified_context, - std::make_shared(header), + Pipe(std::make_shared(header)), SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline()); pipe.addInterpreterContext(modified_context); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 5521b9de39c..eeb5b107b54 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 001684c076d..4264be9dbc2 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index caefa097c3a..709c9dc4a63 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -52,7 +53,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e4682efeaad..e6d41a53bfc 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -25,9 +25,7 @@ #include #include #include -#include #include -#include #include "Processors/Sources/SourceWithProgress.h" #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 32660cb1b1f..d7eef35e60b 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 75ad2761362..79f1d568057 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include #include #include diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 160fc3c2468..7bd8ad2e740 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -21,7 +21,6 @@ #include #include #include -#include #include "registerTableFunctions.h" From 386d47cb227b580a63635e1b94a9d8a765d5cb97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 10:49:12 +0300 Subject: [PATCH 26/89] Fix typo --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 341730bd82d..435bac85bc4 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -4,4 +4,4 @@ SYSTEM FLUSH LOGS; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and databaser=currentDatabase() and type = 'QueryFinish'; +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; From 01ac2fea7991955ad68c8c0a4304fb0649ea84f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 12:44:41 +0300 Subject: [PATCH 27/89] Update 00167_read_bytes_from_fs.sql --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 435bac85bc4..ee3e6b94537 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -4,4 +4,4 @@ SYSTEM FLUSH LOGS; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; From ab28c6c855dfc57b40938f858a99c091896d166e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 13:25:43 +0300 Subject: [PATCH 28/89] Remove BlockInputStream interfaces. --- programs/copier/Internals.cpp | 8 - programs/copier/Internals.h | 2 - programs/library-bridge/Handlers.cpp | 17 +- .../library-bridge/SharedLibraryHandler.h | 1 - programs/odbc-bridge/MainHandler.cpp | 1 - src/Client/IServerConnection.h | 1 - src/Core/ExternalTable.cpp | 1 - src/DataStreams/BlockStreamProfileInfo.cpp | 75 ---- src/DataStreams/BlockStreamProfileInfo.h | 9 - src/DataStreams/IBlockInputStream.cpp | 359 ------------------ src/DataStreams/IBlockInputStream.h | 271 ------------- src/DataStreams/IBlockOutputStream.h | 70 ---- src/DataStreams/IBlockStream_fwd.h | 17 - src/DataStreams/InternalTextLogs.h | 2 +- src/DataStreams/NativeReader.h | 2 +- src/DataStreams/TemporaryFileStream.cpp | 2 - src/DataStreams/TemporaryFileStream.h | 1 - src/DataStreams/copyData.cpp | 86 ----- src/DataStreams/copyData.h | 27 -- .../MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Dictionaries/DictionarySourceHelpers.cpp | 1 - src/Dictionaries/HTTPDictionarySource.cpp | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatFactory.h | 1 - src/Interpreters/Aggregator.h | 1 - src/Interpreters/Context.h | 1 - src/Interpreters/ExpressionAnalyzer.cpp | 1 - src/Interpreters/ExpressionAnalyzer.h | 1 - src/Interpreters/InterpreterExistsQuery.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 1 - src/Interpreters/InterpreterInsertQuery.h | 1 - src/Interpreters/InterpreterSelectQuery.h | 8 - .../InterpreterShowCreateQuery.cpp | 1 - src/Interpreters/InterpreterWatchQuery.cpp | 1 - src/Interpreters/InterpreterWatchQuery.h | 2 - src/Interpreters/JoinSwitcher.h | 1 - src/Interpreters/ProcessList.cpp | 1 - src/Interpreters/SortedBlocksWriter.h | 1 - src/Interpreters/TableJoin.h | 1 - src/Interpreters/executeDDLQueryOnCluster.h | 1 - src/Interpreters/executeQuery.cpp | 2 - .../Formats/OutputStreamToOutputFormat.cpp | 43 --- .../Formats/OutputStreamToOutputFormat.h | 39 -- src/Processors/QueryPipelineBuilder.h | 1 - .../Transforms/CreatingSetsTransform.cpp | 1 - .../Transforms/CreatingSetsTransform.h | 1 - .../Transforms/buildPushingToViewsChain.h | 1 - src/Server/MySQLHandler.cpp | 1 - src/Server/TCPHandler.h | 1 - src/Storages/IStorage.h | 1 - src/Storages/Kafka/StorageKafka.cpp | 1 - src/Storages/LiveView/StorageLiveView.cpp | 1 - .../MergeTree/IMergeTreeDataPartWriter.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 1 - .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 - .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 1 - .../PostgreSQLReplicationHandler.cpp | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 - src/Storages/StorageBuffer.h | 1 - src/Storages/StorageExecutable.cpp | 1 - src/Storages/StorageFile.cpp | 3 - src/Storages/StorageLog.cpp | 2 - src/Storages/StorageMaterializedView.cpp | 1 - src/Storages/StorageMemory.h | 1 - src/Storages/StorageMongoDB.cpp | 1 - src/Storages/StorageMySQL.cpp | 1 - src/Storages/StoragePostgreSQL.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/StorageS3.cpp | 1 - src/Storages/StorageS3Cluster.cpp | 1 - src/Storages/StorageSet.cpp | 1 + src/Storages/StorageStripeLog.cpp | 1 - src/Storages/StorageURL.cpp | 1 - src/Storages/StorageXDBC.cpp | 1 - .../System/StorageSystemZooKeeper.cpp | 1 - src/Storages/tests/gtest_storage_log.cpp | 1 - src/TableFunctions/ITableFunctionFileLike.cpp | 1 - src/TableFunctions/TableFunctionExecutable.h | 1 - 79 files changed, 15 insertions(+), 1090 deletions(-) delete mode 100644 src/DataStreams/IBlockInputStream.cpp delete mode 100644 src/DataStreams/IBlockInputStream.h delete mode 100644 src/DataStreams/IBlockOutputStream.h delete mode 100644 src/DataStreams/IBlockStream_fwd.h delete mode 100644 src/DataStreams/copyData.cpp delete mode 100644 src/DataStreams/copyData.h delete mode 100644 src/Processors/Formats/OutputStreamToOutputFormat.cpp delete mode 100644 src/Processors/Formats/OutputStreamToOutputFormat.h diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index c5e702cd1dc..84283777c8f 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -57,14 +57,6 @@ std::shared_ptr createASTStorageDistributed( } -BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) -{ - return std::make_shared( - stream, - std::numeric_limits::max(), - std::numeric_limits::max()); -} - Block getBlockWithAllStreamData(QueryPipeline pipeline) { QueryPipelineBuilder builder; diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 9e648060592..a9d8ca726fe 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -50,8 +50,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index 2f6dca5ee65..abc5118baad 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -1,7 +1,6 @@ #include "Handlers.h" #include "SharedLibraryHandlerFactory.h" -#include #include #include #include @@ -10,11 +9,13 @@ #include #include #include -#include #include +#include #include #include +#include #include +#include #include #include @@ -189,8 +190,10 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe ReadBufferFromString read_block_buf(params.get("null_values")); auto format = getContext()->getInputFormat(FORMAT, read_block_buf, *sample_block, DEFAULT_BLOCK_SIZE); - auto reader = std::make_shared(format); - auto sample_block_with_nulls = reader->read(); + QueryPipeline pipeline(Pipe(std::move(format))); + PullingPipelineExecutor executor(pipeline); + Block sample_block_with_nulls; + executor.pull(sample_block_with_nulls); LOG_DEBUG(log, "Dictionary sample block with null values: {}", sample_block_with_nulls.dumpStructure()); @@ -281,8 +284,10 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe auto & read_buf = request.getStream(); auto format = getContext()->getInputFormat(FORMAT, read_buf, *requested_sample_block, DEFAULT_BLOCK_SIZE); - auto reader = std::make_shared(format); - auto block = reader->read(); + QueryPipeline pipeline(std::move(format)); + PullingPipelineExecutor executor(pipeline); + Block block; + executor.pull(block); auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); if (!library_handler) diff --git a/programs/library-bridge/SharedLibraryHandler.h b/programs/library-bridge/SharedLibraryHandler.h index f9d2fe43cb2..de1d098dc8d 100644 --- a/programs/library-bridge/SharedLibraryHandler.h +++ b/programs/library-bridge/SharedLibraryHandler.h @@ -2,7 +2,6 @@ #include #include -#include #include "LibraryUtils.h" diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 51abe207095..6a2e967d179 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -4,7 +4,6 @@ #include "ODBCBlockInputStream.h" #include "ODBCBlockOutputStream.h" #include "getIdentifierQuote.h" -#include #include #include #include diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 5584cf72bbf..42886c72182 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -6,7 +6,6 @@ #include #include -#include #include #include diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 7619d60d84e..4dd8b0cf016 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/DataStreams/BlockStreamProfileInfo.cpp index 09ad8a8e4ac..9a06d905223 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/DataStreams/BlockStreamProfileInfo.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -47,16 +46,12 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk size_t BlockStreamProfileInfo::getRowsBeforeLimit() const { - if (!calculated_rows_before_limit) - calculateRowsBeforeLimit(); return rows_before_limit; } bool BlockStreamProfileInfo::hasAppliedLimit() const { - if (!calculated_rows_before_limit) - calculateRowsBeforeLimit(); return applied_limit; } @@ -73,74 +68,4 @@ void BlockStreamProfileInfo::update(size_t num_rows, size_t num_bytes) bytes += num_bytes; } - -void BlockStreamProfileInfo::collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const -{ - if (!parent) - return; - - if (parent->getName() == name) - { - res.push_back(this); - return; - } - - parent->forEachChild([&] (IBlockInputStream & child) - { - child.getProfileInfo().collectInfosForStreamsWithName(name, res); - return false; - }); -} - - -void BlockStreamProfileInfo::calculateRowsBeforeLimit() const -{ - calculated_rows_before_limit = true; - - /// is there a Limit? - BlockStreamProfileInfos limits; - collectInfosForStreamsWithName("Limit", limits); - - if (!limits.empty()) - { - applied_limit = true; - - /** Take the number of lines read below `PartialSorting`, if any, or below `Limit`. - * This is necessary, because sorting can return only part of the rows. - */ - BlockStreamProfileInfos partial_sortings; - collectInfosForStreamsWithName("PartialSorting", partial_sortings); - - BlockStreamProfileInfos & limits_or_sortings = partial_sortings.empty() ? limits : partial_sortings; - - for (const BlockStreamProfileInfo * info_limit_or_sort : limits_or_sortings) - { - info_limit_or_sort->parent->forEachChild([&] (IBlockInputStream & child) - { - rows_before_limit += child.getProfileInfo().rows; - return false; - }); - } - } - else - { - /// Then the data about `rows_before_limit` can be in `RemoteBlockInputStream` (come from a remote server). - BlockStreamProfileInfos remotes; - collectInfosForStreamsWithName("Remote", remotes); - collectInfosForStreamsWithName("TreeExecutor", remotes); - - if (remotes.empty()) - return; - - for (const auto & info : remotes) - { - if (info->applied_limit) - { - applied_limit = true; - rows_before_limit += info->rows_before_limit; - } - } - } -} - } diff --git a/src/DataStreams/BlockStreamProfileInfo.h b/src/DataStreams/BlockStreamProfileInfo.h index 688bdfc91fc..1707b941445 100644 --- a/src/DataStreams/BlockStreamProfileInfo.h +++ b/src/DataStreams/BlockStreamProfileInfo.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -16,9 +15,6 @@ class WriteBuffer; /// Information for profiling. See IBlockInputStream.h struct BlockStreamProfileInfo { - /// Info about stream object this profile info refers to. - IBlockInputStream * parent = nullptr; - bool started = false; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time @@ -28,9 +24,6 @@ struct BlockStreamProfileInfo using BlockStreamProfileInfos = std::vector; - /// Collect BlockStreamProfileInfo for the nearest sources in the tree named `name`. Example; collect all info for PartialSorting streams. - void collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const; - /** Get the number of rows if there were no LIMIT. * If there is no LIMIT, 0 is returned. * If the query does not contain ORDER BY, the number can be underestimated - return the number of rows in blocks that were read before LIMIT reached. @@ -59,8 +52,6 @@ struct BlockStreamProfileInfo } private: - void calculateRowsBeforeLimit() const; - /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp deleted file mode 100644 index e57d6903673..00000000000 --- a/src/DataStreams/IBlockInputStream.cpp +++ /dev/null @@ -1,359 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; - extern const Event SelectedRows; - extern const Event SelectedBytes; -} - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int QUERY_WAS_CANCELLED; - extern const int TOO_MANY_ROWS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int LOGICAL_ERROR; -} - - -/// It's safe to access children without mutex as long as these methods are called before first call to `read()` or `readPrefix()`. - - -Block IBlockInputStream::read() -{ - if (total_rows_approx) - { - progressImpl(Progress(0, 0, total_rows_approx)); - total_rows_approx = 0; - } - - if (!info.started) - { - info.total_stopwatch.start(); - info.started = true; - } - - Block res; - - if (isCancelledOrThrowIfKilled()) - return res; - - if (!checkTimeLimit()) - limit_exceeded_need_break = true; - - if (!limit_exceeded_need_break) - res = readImpl(); - - if (res) - { - info.update(res); - - if (enabled_extremes) - updateExtremes(res); - - if (limits.mode == LimitsMode::LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) - limit_exceeded_need_break = true; - - if (quota) - checkQuota(res); - } - else - { - /** If the stream is over, then we will ask all children to abort the execution. - * This makes sense when running a query with LIMIT - * - there is a situation when all the necessary data has already been read, - * but children sources are still working, - * herewith they can work in separate threads or even remotely. - */ - cancel(false); - } - - progress(Progress(res.rows(), res.bytes())); - -#ifndef NDEBUG - if (res) - { - Block header = getHeader(); - if (header) - assertBlocksHaveEqualStructure(res, header, getName()); - } -#endif - - return res; -} - - -void IBlockInputStream::readPrefix() -{ -#ifndef NDEBUG - if (!read_prefix_is_called) - read_prefix_is_called = true; - else - throw Exception("readPrefix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - readPrefixImpl(); - - forEachChild([&] (IBlockInputStream & child) - { - child.readPrefix(); - return false; - }); -} - - -void IBlockInputStream::readSuffix() -{ -#ifndef NDEBUG - if (!read_suffix_is_called) - read_suffix_is_called = true; - else - throw Exception("readSuffix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - forEachChild([&] (IBlockInputStream & child) - { - child.readSuffix(); - return false; - }); - - readSuffixImpl(); -} - - -void IBlockInputStream::updateExtremes(Block & block) -{ - size_t num_columns = block.columns(); - - if (!extremes) - { - MutableColumns extremes_columns(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnPtr & src = block.safeGetByPosition(i).column; - - if (isColumnConst(*src)) - { - /// Equal min and max. - extremes_columns[i] = src->cloneResized(2); - } - else - { - Field min_value; - Field max_value; - - src->getExtremes(min_value, max_value); - - extremes_columns[i] = src->cloneEmpty(); - - extremes_columns[i]->insert(min_value); - extremes_columns[i]->insert(max_value); - } - } - - extremes = block.cloneWithColumns(std::move(extremes_columns)); - } - else - { - for (size_t i = 0; i < num_columns; ++i) - { - ColumnPtr & old_extremes = extremes.safeGetByPosition(i).column; - - if (isColumnConst(*old_extremes)) - continue; - - Field min_value = (*old_extremes)[0]; - Field max_value = (*old_extremes)[1]; - - Field cur_min_value; - Field cur_max_value; - - block.safeGetByPosition(i).column->getExtremes(cur_min_value, cur_max_value); - - if (cur_min_value < min_value) - min_value = cur_min_value; - if (cur_max_value > max_value) - max_value = cur_max_value; - - MutableColumnPtr new_extremes = old_extremes->cloneEmpty(); - - new_extremes->insert(min_value); - new_extremes->insert(max_value); - - old_extremes = std::move(new_extremes); - } - } -} - - -bool IBlockInputStream::checkTimeLimit() const -{ - return limits.speed_limits.checkTimeLimit(info.total_stopwatch, limits.timeout_overflow_mode); -} - - -void IBlockInputStream::checkQuota(Block & block) -{ - switch (limits.mode) - { - case LimitsMode::LIMITS_TOTAL: - /// Checked in `progress` method. - break; - - case LimitsMode::LIMITS_CURRENT: - { - UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); - quota->used({Quota::RESULT_ROWS, block.rows()}, {Quota::RESULT_BYTES, block.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); - prev_elapsed = total_elapsed; - break; - } - } -} - - -void IBlockInputStream::progressImpl(const Progress & value) -{ - if (progress_callback) - progress_callback(value); - - if (process_list_elem) - { - if (!process_list_elem->updateProgressIn(value)) - cancel(/* kill */ true); - - /// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers. - - ProgressValues progress = process_list_elem->getProgressIn(); - size_t total_rows_estimate = std::max(progress.read_rows, progress.total_rows_to_read); - - /** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read. - * NOTE: Maybe it makes sense to have them checked directly in ProcessList? - */ - if (limits.mode == LimitsMode::LIMITS_TOTAL) - { - if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read", - ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) - cancel(false); - } - - size_t total_rows = progress.total_rows_to_read; - - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); - - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } - - limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); - - if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) - quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); - } - - ProfileEvents::increment(ProfileEvents::SelectedRows, value.read_rows); - ProfileEvents::increment(ProfileEvents::SelectedBytes, value.read_bytes); -} - - -void IBlockInputStream::cancel(bool kill) -{ - if (kill) - is_killed = true; - - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) - return; - - forEachChild([&] (IBlockInputStream & child) - { - child.cancel(kill); - return false; - }); -} - - -bool IBlockInputStream::isCancelled() const -{ - return is_cancelled; -} - -bool IBlockInputStream::isCancelledOrThrowIfKilled() const -{ - if (!is_cancelled) - return false; - if (is_killed) - throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); - return true; -} - - -void IBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - progress_callback = callback; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProgressCallback(callback); - return false; - }); -} - - -void IBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - process_list_elem = elem; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProcessListElement(elem); - return false; - }); -} - - -Block IBlockInputStream::getTotals() -{ - if (totals) - return totals; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getTotals(); - return bool(res); - }); - return res; -} - - -Block IBlockInputStream::getExtremes() -{ - if (extremes) - return extremes; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getExtremes(); - return bool(res); - }); - return res; -} - -} diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h deleted file mode 100644 index 0e77ba81779..00000000000 --- a/src/DataStreams/IBlockInputStream.h +++ /dev/null @@ -1,271 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ -} - -class ProcessListElement; -class EnabledQuota; -class QueryStatus; - - -/** The stream interface for reading data by blocks from the database. - * Relational operations are supposed to be done also as implementations of this interface. - * Watches out at how the source of the blocks works. - * Lets you get information for profiling: rows per second, blocks per second, megabytes per second, etc. - * Allows you to stop reading data (in nested sources). - */ -class IBlockInputStream : public TypePromotion -{ - friend struct BlockStreamProfileInfo; - -public: - IBlockInputStream() { info.parent = this; } - virtual ~IBlockInputStream() = default; - - IBlockInputStream(const IBlockInputStream &) = delete; - IBlockInputStream & operator=(const IBlockInputStream &) = delete; - - /// To output the data stream transformation tree (query execution plan). - virtual String getName() const = 0; - - /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). - * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. - * It is guaranteed that method "read" returns blocks of exactly that structure. - */ - virtual Block getHeader() const = 0; - - virtual const BlockMissingValues & getMissingValues() const - { - static const BlockMissingValues none; - return none; - } - - /** Read next block. - * If there are no more blocks, return an empty block (for which operator `bool` returns false). - * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. - * This also applies for readPrefix, readSuffix. - */ - Block read(); - - /** Read something before starting all data or after the end of all data. - * In the `readSuffix` function, you can implement a finalization that can lead to an exception. - * readPrefix() must be called before the first call to read(). - * readSuffix() should be called after read() returns an empty block, or after a call to cancel(), but not during read() execution. - */ - - /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children. - * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function, - * but you want them to be called, for example, in separate threads (for parallel initialization of children). - * Then overload `readPrefix` function. - */ - virtual void readPrefix(); - - /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself. - * If this stream calls read() in children in a separate thread, this behavior is usually incorrect: - * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread. - * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams. - */ - virtual void readSuffix(); - - /// Do not allow to change the table while the blocks stream and its children are alive. - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - - /// Get information about execution speed. - const BlockStreamProfileInfo & getProfileInfo() const { return info; } - - /** Get "total" values. - * The default implementation takes them from itself or from the first child source in which they are. - * The overridden method can perform some calculations. For example, apply an expression to the `totals` of the child source. - * There can be no total values - then an empty block is returned. - * - * Call this method only after all the data has been retrieved with `read`, - * otherwise there will be problems if any data at the same time is computed in another thread. - */ - virtual Block getTotals(); - - /// The same for minimums and maximums. - virtual Block getExtremes(); - - - /** Set the execution progress bar callback. - * The callback is passed to all child sources. - * By default, it is called for leaf sources, after each block. - * (But this can be overridden in the progress() method) - * The function takes the number of rows in the last block, the number of bytes in the last block. - * Note that the callback can be called from different threads. - */ - virtual void setProgressCallback(const ProgressCallback & callback); - - - /** In this method: - * - the progress callback is called; - * - the status of the query execution in ProcessList is updated; - * - checks restrictions and quotas that should be checked not within the same source, - * but over the total amount of resources spent in all sources at once (information in the ProcessList). - */ - virtual void progress(const Progress & value) - { - /// The data for progress is taken from leaf sources. - if (children.empty()) - progressImpl(value); - } - - void progressImpl(const Progress & value); - - - /** Set the pointer to the process list item. - * It is passed to all child sources. - * General information about the resources spent on the request will be written into it. - * Based on this information, the quota and some restrictions will be checked. - * This information will also be available in the SHOW PROCESSLIST request. - */ - virtual void setProcessListElement(QueryStatus * elem); - - /** Set the approximate total number of rows to read. - */ - void addTotalRowsApprox(size_t value) { total_rows_approx += value; } - - - /** Ask to abort the receipt of data as soon as possible. - * By default - just sets the flag is_cancelled and asks that all children be interrupted. - * This function can be called several times, including simultaneously from different threads. - * Have two modes: - * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data. - * with kill = true also is_killed set - queries will stop with exception. - */ - virtual void cancel(bool kill); - - bool isCancelled() const; - bool isCancelledOrThrowIfKilled() const; - - /** Set limitations that checked on each block. */ - virtual void setLimits(const StreamLocalLimits & limits_) - { - limits = limits_; - } - - const StreamLocalLimits & getLimits() const - { - return limits; - } - - /** Set the quota. If you set a quota on the amount of raw data, - * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - */ - virtual void setQuota(const std::shared_ptr & new_quota) - { - quota = new_quota; - } - - /// Enable calculation of minimums and maximums by the result columns. - void enableExtremes() { enabled_extremes = true; } - -protected: - /// Order is important: `table_locks` must be destroyed after `children` so that tables from - /// which child streams read are protected by the locks during the lifetime of the child streams. - std::vector table_locks; - - BlockInputStreams children; - std::shared_mutex children_mutex; - - BlockStreamProfileInfo info; - std::atomic is_cancelled{false}; - std::atomic is_killed{false}; - ProgressCallback progress_callback; - QueryStatus * process_list_elem = nullptr; - /// According to total_stopwatch in microseconds - UInt64 last_profile_events_update_time = 0; - - /// Additional information that can be generated during the work process. - - /// Total values during aggregation. - Block totals; - /// Minimums and maximums. The first row of the block - minimums, the second - the maximums. - Block extremes; - - - void addChild(const BlockInputStreamPtr & child) - { - std::unique_lock lock(children_mutex); - children.push_back(child); - } - - /** Check limits. - * But only those that can be checked within each separate stream. - */ - bool checkTimeLimit() const; - -#ifndef NDEBUG - bool read_prefix_is_called = false; - bool read_suffix_is_called = false; -#endif - -private: - bool enabled_extremes = false; - - /// The limit on the number of rows/bytes has been exceeded, and you need to stop execution on the next `read` call, as if the thread has run out. - bool limit_exceeded_need_break = false; - - /// Limitations and quotas. - - StreamLocalLimits limits; - - std::shared_ptr quota; /// If nullptr - the quota is not used. - UInt64 prev_elapsed = 0; - - /// The approximate total number of rows to read. For progress bar. - size_t total_rows_approx = 0; - - /// Derived classes must implement this function. - virtual Block readImpl() = 0; - - /// Here you can do a preliminary initialization. - virtual void readPrefixImpl() {} - - /// Here you need to do a finalization, which can lead to an exception. - virtual void readSuffixImpl() {} - - void updateExtremes(Block & block); - - /** Check quotas. - * But only those that can be checked within each separate stream. - */ - void checkQuota(Block & block); - - size_t checkDepthImpl(size_t max_depth, size_t level) const; - - template - void forEachChild(F && f) - { - /// NOTE: Acquire a read lock, therefore f() should be thread safe - std::shared_lock lock(children_mutex); - - // Reduce lock scope and avoid recursive locking since that is undefined for shared_mutex. - const auto children_copy = children; - lock.unlock(); - - for (auto & child : children_copy) - if (f(*child)) - return; - } - -}; - -} diff --git a/src/DataStreams/IBlockOutputStream.h b/src/DataStreams/IBlockOutputStream.h deleted file mode 100644 index 65ebd90769d..00000000000 --- a/src/DataStreams/IBlockOutputStream.h +++ /dev/null @@ -1,70 +0,0 @@ -#pragma once - -#include -#include -#include - -#include - -#include -#include -#include - - -namespace DB -{ - -struct Progress; - -/** Interface of stream for writing data (into table, filesystem, network, terminal, etc.) - */ -class IBlockOutputStream : private boost::noncopyable -{ -public: - IBlockOutputStream() = default; - - /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). - * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. - * You must pass blocks of exactly this structure to the 'write' method. - */ - virtual Block getHeader() const = 0; - - /** Write block. - */ - virtual void write(const Block & block) = 0; - - /** Write or do something before all data or after all data. - */ - virtual void writePrefix() {} - virtual void writeSuffix() {} - - /** Flush output buffers if any. - */ - virtual void flush() {} - - /** Methods to set additional information for output in formats, that support it. - */ - virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} - virtual void setTotals(const Block & /*totals*/) {} - virtual void setExtremes(const Block & /*extremes*/) {} - - /** Notify about progress. Method could be called from different threads. - * Passed value are delta, that must be summarized. - */ - virtual void onProgress(const Progress & /*progress*/) {} - - /** Content-Type to set when sending HTTP response. - */ - virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; } - - virtual ~IBlockOutputStream() = default; - - /** Don't let to alter table while instance of stream is alive. - */ - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - -private: - std::vector table_locks; -}; - -} diff --git a/src/DataStreams/IBlockStream_fwd.h b/src/DataStreams/IBlockStream_fwd.h deleted file mode 100644 index d74a9528ed9..00000000000 --- a/src/DataStreams/IBlockStream_fwd.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class IBlockInputStream; -class IBlockOutputStream; - -using BlockInputStreamPtr = std::shared_ptr; -using BlockInputStreams = std::vector; -using BlockOutputStreamPtr = std::shared_ptr; -using BlockOutputStreams = std::vector; - -} diff --git a/src/DataStreams/InternalTextLogs.h b/src/DataStreams/InternalTextLogs.h index 1312c1d327c..a8b119b0f69 100644 --- a/src/DataStreams/InternalTextLogs.h +++ b/src/DataStreams/InternalTextLogs.h @@ -1,6 +1,6 @@ #pragma once -#include #include +#include namespace DB diff --git a/src/DataStreams/NativeReader.h b/src/DataStreams/NativeReader.h index cfd58bde2cc..95b03c71764 100644 --- a/src/DataStreams/NativeReader.h +++ b/src/DataStreams/NativeReader.h @@ -1,8 +1,8 @@ #pragma once -#include #include #include +#include namespace DB { diff --git a/src/DataStreams/TemporaryFileStream.cpp b/src/DataStreams/TemporaryFileStream.cpp index 826cf5508d8..4b7c9d50fe7 100644 --- a/src/DataStreams/TemporaryFileStream.cpp +++ b/src/DataStreams/TemporaryFileStream.cpp @@ -1,8 +1,6 @@ #include -#include #include #include -#include #include #include #include diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index c0c13605928..e288b5b30fa 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/DataStreams/copyData.cpp b/src/DataStreams/copyData.cpp deleted file mode 100644 index a26052778a8..00000000000 --- a/src/DataStreams/copyData.cpp +++ /dev/null @@ -1,86 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -bool isAtomicSet(std::atomic * val) -{ - return ((val != nullptr) && val->load(std::memory_order_seq_cst)); -} - -} - -template -void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCallback && is_cancelled, TProgressCallback && progress) -{ - from.readPrefix(); - to.writePrefix(); - - while (Block block = from.read()) - { - if (is_cancelled()) - break; - - to.write(block); - progress(block); - } - - if (is_cancelled()) - return; - - /// For outputting additional information in some formats. - if (from.getProfileInfo().hasAppliedLimit()) - to.setRowsBeforeLimit(from.getProfileInfo().getRowsBeforeLimit()); - - to.setTotals(from.getTotals()); - to.setExtremes(from.getExtremes()); - - if (is_cancelled()) - return; - - from.readSuffix(); - to.writeSuffix(); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & progress, - std::atomic * is_cancelled) -{ - auto is_cancelled_pred = [is_cancelled] () - { - return isAtomicSet(is_cancelled); - }; - - copyDataImpl(from, to, is_cancelled_pred, progress); -} - -inline void doNothing(const Block &) {} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled) -{ - auto is_cancelled_pred = [is_cancelled] () - { - return isAtomicSet(is_cancelled); - }; - - copyDataImpl(from, to, is_cancelled_pred, doNothing); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled) -{ - copyDataImpl(from, to, is_cancelled, doNothing); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled, - const std::function & progress) -{ - copyDataImpl(from, to, is_cancelled, progress); -} - -} diff --git a/src/DataStreams/copyData.h b/src/DataStreams/copyData.h deleted file mode 100644 index 3dc90aed37d..00000000000 --- a/src/DataStreams/copyData.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include - -#include -#include - - -namespace DB -{ - -class Block; - -/** Copies data from the InputStream into the OutputStream - * (for example, from the database to the console, etc.) - */ -void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled = nullptr); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & progress, - std::atomic * is_cancelled = nullptr); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled, - const std::function & progress); - -} diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index f4a5b6b5e4e..9ec8a9523c6 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -14,7 +14,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index cf003dceb8e..cd87cf831a2 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -1,7 +1,6 @@ #include "DictionarySourceHelpers.h" #include #include -#include #include #include #include "DictionaryStructure.h" diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 6cb4d52744a..844a6357e29 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,5 +1,4 @@ #include "HTTPDictionarySource.h" -#include #include #include #include diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ed9f9d52b94..34574ca13f8 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d816ef1d016..f20cec56943 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 85ce83868c6..975075eba96 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -19,7 +19,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1e19c18de43..22ae459a662 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 566ee60a3e6..9d770aee159 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -35,7 +35,6 @@ #include #include -#include #include diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index c785b085a57..b6bb3c5fad5 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 6eb188bce9f..6ffeef5cc7d 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 8a31917caef..6a1a8652b23 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 0d6fe34c0c2..5f44603a420 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 21e15bc74bb..cf24d14b737 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -52,13 +51,6 @@ public: const SelectQueryOptions &, const Names & required_result_column_names_ = Names{}); - /// Read data not from the table specified in the query, but from the prepared source `input`. - InterpreterSelectQuery( - const ASTPtr & query_ptr_, - ContextPtr context_, - const BlockInputStreamPtr & input_, - const SelectQueryOptions & = {}); - /// Read data not from the table specified in the query, but from the prepared pipe `input`. InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index adf1aae3ff3..c191a73bc71 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index bc0aeda56bd..78c4eca5ca6 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -15,7 +15,6 @@ limitations under the License. */ #include #include #include -#include #include diff --git a/src/Interpreters/InterpreterWatchQuery.h b/src/Interpreters/InterpreterWatchQuery.h index 2bc7236582a..e43ed88af2f 100644 --- a/src/Interpreters/InterpreterWatchQuery.h +++ b/src/Interpreters/InterpreterWatchQuery.h @@ -38,8 +38,6 @@ private: /// Table from where to read data, if not subquery. StoragePtr storage; - /// Streams of read data - BlockInputStreams streams; }; } diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index aaa7441b8a4..30115710e22 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index f8402cf0287..fb9f2e25c07 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index c65511e943e..94bebce88f7 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 22cd958f4f8..02dcd95ab41 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index f430c2364b2..0ad40dd3332 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace zkutil diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3f5b386d16f..0a1130c721b 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -10,8 +10,6 @@ #include #include -#include -#include #include #include diff --git a/src/Processors/Formats/OutputStreamToOutputFormat.cpp b/src/Processors/Formats/OutputStreamToOutputFormat.cpp deleted file mode 100644 index 5d4e7832327..00000000000 --- a/src/Processors/Formats/OutputStreamToOutputFormat.cpp +++ /dev/null @@ -1,43 +0,0 @@ -#include -#include - -namespace DB -{ - -Block OutputStreamToOutputFormat::getHeader() const -{ - return output_format->getPort(IOutputFormat::PortKind::Main).getHeader(); -} - -void OutputStreamToOutputFormat::write(const Block & block) -{ - output_format->write(block); -} - -void OutputStreamToOutputFormat::writePrefix() { output_format->doWritePrefix(); } -void OutputStreamToOutputFormat::writeSuffix() { output_format->doWriteSuffix(); } - -void OutputStreamToOutputFormat::flush() { output_format->flush(); } - -void OutputStreamToOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) -{ - output_format->setRowsBeforeLimit(rows_before_limit); -} - -void OutputStreamToOutputFormat::setTotals(const Block & totals) -{ - if (totals) - output_format->setTotals(totals); -} - -void OutputStreamToOutputFormat::setExtremes(const Block & extremes) -{ - if (extremes) - output_format->setExtremes(extremes); -} - -void OutputStreamToOutputFormat::onProgress(const Progress & progress) { output_format->onProgress(progress); } - -std::string OutputStreamToOutputFormat::getContentType() const { return output_format->getContentType(); } - -} diff --git a/src/Processors/Formats/OutputStreamToOutputFormat.h b/src/Processors/Formats/OutputStreamToOutputFormat.h deleted file mode 100644 index a85de12b49d..00000000000 --- a/src/Processors/Formats/OutputStreamToOutputFormat.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once -#include - -namespace DB -{ - - -class IOutputFormat; - -using OutputFormatPtr = std::shared_ptr; - -/// Wrapper. Implements IBlockOutputStream interface using IOutputFormat object. -class OutputStreamToOutputFormat : public IBlockOutputStream -{ -public: - explicit OutputStreamToOutputFormat(OutputFormatPtr output_format_) : output_format(std::move(output_format_)) {} - - Block getHeader() const override; - - void write(const Block & block) override; - - void writePrefix() override; - void writeSuffix() override; - - void flush() override; - - void setRowsBeforeLimit(size_t rows_before_limit) override; - void setTotals(const Block & totals) override; - void setExtremes(const Block & extremes) override; - - void onProgress(const Progress & progress) override; - - std::string getContentType() const override; - -private: - OutputFormatPtr output_format; -}; - -} diff --git a/src/Processors/QueryPipelineBuilder.h b/src/Processors/QueryPipelineBuilder.h index 78ae5dd41be..7e0ddbc0285 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/Processors/QueryPipelineBuilder.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6776caae9bf..e6ae620e69b 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index eca12c33f54..a5a67e99afc 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 75d0528ff7b..6956dedbc41 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 3720362775c..8f4f04e56c5 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index d001b12ee66..e89d82cfcc8 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -13,7 +13,6 @@ #include #include #include -#include #include "IServer.h" diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..4ed3a43d2ed 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 690c9cbd4d0..39688060b0a 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 4641a1631f2..be3dd9ae6c9 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,7 +21,6 @@ limitations under the License. */ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 5e12d5da678..d0d3f283478 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f4adee8c259..5dbca837f31 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index d952950e461..e64ba9edec0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 75308f872dc..e3ca902b1bd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -2,9 +2,9 @@ #include #include #include -#include #include #include +#include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 9e138e9882a..947c0bbe932 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -3,7 +3,6 @@ #include "StorageMaterializedPostgreSQL.h" #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 873a4b4860c..3796bd8ba57 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 2b4f5e4a276..cf9b557de25 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 59f250d67b8..3e8955ad864 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index d6e242d1a97..0f47f654428 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9aa5689aa66..4ae55272db6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -16,7 +16,6 @@ #include #include -#include #include #include @@ -479,8 +478,6 @@ Pipe StorageFile::read( size_t max_block_size, unsigned num_streams) { - BlockInputStreams blocks_input; - if (use_table_fd) /// need to call ctr BlockInputStream paths = {""}; /// when use fd, paths are empty else diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..f0c4509f188 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -16,8 +16,6 @@ #include -#include - #include #include diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 5a9e8fc2461..904d1a7f89c 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 846fd4af5fd..063802faf1a 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -8,7 +8,6 @@ #include #include -#include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index eeb5b107b54..15430f60285 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 4264be9dbc2..3bdf3218b2e 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index b4bb5400930..10a60bf9b21 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..20e64255684 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -60,7 +60,6 @@ #include #include -#include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 709c9dc4a63..d95a9465bd6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -30,7 +30,6 @@ #include -#include #include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e6d41a53bfc..0e7faad194e 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 58c56f1401f..2547af1b0ad 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 2dc2577f245..0cd07afc26c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 79f1d568057..9397986fadd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -15,7 +15,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a1254e2aaeb..a0924896437 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -1,6 +1,5 @@ #include "StorageXDBC.h" -#include #include #include #include diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 9aedee66b5f..f2b2102c7ff 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index e7ecfc7c4f0..5b891c43aae 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index afd81638da4..49461fe8f46 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -16,7 +16,6 @@ #include -#include namespace DB diff --git a/src/TableFunctions/TableFunctionExecutable.h b/src/TableFunctions/TableFunctionExecutable.h index 05ef2b3b26b..128ee8e46fc 100644 --- a/src/TableFunctions/TableFunctionExecutable.h +++ b/src/TableFunctions/TableFunctionExecutable.h @@ -1,6 +1,5 @@ #pragma once -#include #include namespace DB From 2da43012b6203bd9957d099dfbff575420efa4af Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 30 Sep 2021 22:46:12 +0300 Subject: [PATCH 29/89] Add log levels updates --- base/loggers/Loggers.cpp | 91 +++++++++++++++++++++++++++-- base/loggers/Loggers.h | 2 + base/loggers/OwnFormattingChannel.h | 5 +- base/loggers/OwnSplitChannel.cpp | 17 +++++- base/loggers/OwnSplitChannel.h | 6 +- programs/server/Server.cpp | 2 +- 6 files changed, 111 insertions(+), 12 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 80e62d0a6d6..0f41296819e 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -84,7 +84,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, log_file); log->setLevel(log_level); - split->addChannel(log); + split->addChannel(log, "log"); } const auto errorlog_path = config.getString("logger.errorlog", ""); @@ -116,7 +116,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr errorlog = new DB::OwnFormattingChannel(pf, error_log_file); errorlog->setLevel(errorlog_level); errorlog->open(); - split->addChannel(errorlog); + split->addChannel(errorlog, "errorlog"); } if (config.getBool("logger.use_syslog", false)) @@ -155,7 +155,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, syslog_channel); log->setLevel(syslog_level); - split->addChannel(log); + split->addChannel(log, "syslog"); } bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO); @@ -177,7 +177,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel); logger.warning("Logging " + console_log_level_string + " to console"); log->setLevel(console_log_level); - split->addChannel(log); + split->addChannel(log, "console"); } split->open(); @@ -224,6 +224,89 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log } } +void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger) +{ + int max_log_level = 0; + + const auto log_level_string = config.getString("logger.level", "trace"); + int log_level = Poco::Logger::parseLevel(log_level_string); + if (log_level > max_log_level) + max_log_level = log_level; + + const auto log_path = config.getString("logger.log", ""); + if (!log_path.empty()) + split->setLevel("log", log_level); + else + split->setLevel("log", 0); + + // Set level to console + bool is_daemon = config.getBool("application.runAsDaemon", false); + bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO); + if (config.getBool("logger.console", false) + || (!config.hasProperty("logger.console") && !is_daemon && should_log_to_console)) + split->setLevel("console", log_level); + else + split->setLevel("console", 0); + + // Set level to errorlog + int errorlog_level = 0; + const auto errorlog_path = config.getString("logger.errorlog", ""); + if (!errorlog_path.empty()) + { + errorlog_level = Poco::Logger::parseLevel(config.getString("logger.errorlog_level", "notice")); + if (errorlog_level > max_log_level) + max_log_level = errorlog_level; + } + split->setLevel("errorlog", errorlog_level); + + // Set level to syslog + int syslog_level = 0; + if (config.getBool("logger.use_syslog", false)) + { + syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level_string)); + if (syslog_level > max_log_level) + max_log_level = syslog_level; + } + split->setLevel("syslog", syslog_level); + + // Global logging level (it can be overridden for specific loggers). + logger.setLevel(max_log_level); + + // Set level to all already created loggers + std::vector names; + + logger.root().names(names); + for (const auto & name : names) + logger.root().get(name).setLevel(max_log_level); + + logger.root().setLevel(max_log_level); + + // Explicitly specified log levels for specific loggers. + { + Poco::Util::AbstractConfiguration::Keys loggers_level; + config.keys("logger.levels", loggers_level); + + if (!loggers_level.empty()) + { + for (const auto & key : loggers_level) + { + if (key == "logger" || key.starts_with("logger[")) + { + const std::string name(config.getString("logger.levels." + key + ".name")); + const std::string level(config.getString("logger.levels." + key + ".level")); + logger.root().get(name).setLevel(level); + } + else + { + // Legacy syntax + const std::string level(config.getString("logger.levels." + key, "trace")); + logger.root().get(key).setLevel(level); + } + } + } + } +} + void Loggers::closeLogs(Poco::Logger & logger) { if (log_file) diff --git a/base/loggers/Loggers.h b/base/loggers/Loggers.h index 151c1d3566f..e8afd749534 100644 --- a/base/loggers/Loggers.h +++ b/base/loggers/Loggers.h @@ -19,6 +19,8 @@ class Loggers public: void buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger, const std::string & cmd_name = ""); + void updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger); + /// Close log files. On next log write files will be reopened. void closeLogs(Poco::Logger & logger); diff --git a/base/loggers/OwnFormattingChannel.h b/base/loggers/OwnFormattingChannel.h index 2336dacad04..0480d0d5061 100644 --- a/base/loggers/OwnFormattingChannel.h +++ b/base/loggers/OwnFormattingChannel.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include #include @@ -14,7 +15,7 @@ class OwnFormattingChannel : public Poco::Channel, public ExtendedLogChannel public: explicit OwnFormattingChannel( Poco::AutoPtr pFormatter_ = nullptr, Poco::AutoPtr pChannel_ = nullptr) - : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)) + : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)), priority(Poco::Message::PRIO_TRACE) { } @@ -45,7 +46,7 @@ public: private: Poco::AutoPtr pFormatter; Poco::AutoPtr pChannel; - Poco::Message::Priority priority = Poco::Message::PRIO_TRACE; + std::atomic priority; }; } diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 2349c60856f..2ae1e65729c 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -1,4 +1,5 @@ #include "OwnSplitChannel.h" +#include "OwnFormattingChannel.h" #include #include @@ -75,7 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg); /// Log data to child channels - for (auto & channel : channels) + for (auto & [name, channel] : channels) { if (channel.second) channel.second->logExtended(msg_ext); // extended child @@ -137,9 +138,9 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) } -void OwnSplitChannel::addChannel(Poco::AutoPtr channel) +void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std::string & name) { - channels.emplace_back(std::move(channel), dynamic_cast(channel.get())); + channels.emplace(name, ExtendedChannelPtrPair(std::move(channel), dynamic_cast(channel.get()))); } void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_priority) @@ -149,4 +150,14 @@ void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_prior text_log_max_priority.store(max_priority, std::memory_order_relaxed); } +void OwnSplitChannel::setLevel(const std::string & name, int level) +{ + auto it = channels.find(name); + if (it != channels.end()) + { + if (auto * channel = dynamic_cast(it->second.first.get())) + channel->setLevel(level); + } +} + } diff --git a/base/loggers/OwnSplitChannel.h b/base/loggers/OwnSplitChannel.h index 03ff7b57745..fdc580e65f8 100644 --- a/base/loggers/OwnSplitChannel.h +++ b/base/loggers/OwnSplitChannel.h @@ -18,10 +18,12 @@ public: /// Makes an extended message from msg and passes it to the client logs queue and child (if possible) void log(const Poco::Message & msg) override; /// Adds a child channel - void addChannel(Poco::AutoPtr channel); + void addChannel(Poco::AutoPtr channel, const std::string & name); void addTextLog(std::shared_ptr log, int max_priority); + void setLevel(const std::string & name, int level); + private: void logSplit(const Poco::Message & msg); void tryLogSplit(const Poco::Message & msg); @@ -29,7 +31,7 @@ private: using ChannelPtr = Poco::AutoPtr; /// Handler and its pointer casted to extended interface using ExtendedChannelPtrPair = std::pair; - std::vector channels; + std::map channels; std::mutex text_log_mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 01033570926..bfb77f49763 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -844,7 +844,7 @@ if (ThreadFuzzer::instance().isEffective()) // FIXME logging-related things need synchronization -- see the 'Logger * log' saved // in a lot of places. For now, disable updating log configuration without server restart. //setTextLog(global_context->getTextLog()); - //buildLoggers(*config, logger()); + updateLevels(*config, logger()); global_context->setClustersConfig(config); global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); From f29261741c2dcc45866a76b89a9176abb1482f44 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 4 Oct 2021 12:25:42 +0300 Subject: [PATCH 30/89] Add test --- .../test_log_levels_update/__init__.py | 0 .../test_log_levels_update/configs/log.xml | 6 +++ .../test_log_levels_update/test.py | 50 +++++++++++++++++++ 3 files changed, 56 insertions(+) create mode 100644 tests/integration/test_log_levels_update/__init__.py create mode 100644 tests/integration/test_log_levels_update/configs/log.xml create mode 100644 tests/integration/test_log_levels_update/test.py diff --git a/tests/integration/test_log_levels_update/__init__.py b/tests/integration/test_log_levels_update/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_log_levels_update/configs/log.xml b/tests/integration/test_log_levels_update/configs/log.xml new file mode 100644 index 00000000000..668a15f6afd --- /dev/null +++ b/tests/integration/test_log_levels_update/configs/log.xml @@ -0,0 +1,6 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + + \ No newline at end of file diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py new file mode 100644 index 00000000000..dca660a2982 --- /dev/null +++ b/tests/integration/test_log_levels_update/test.py @@ -0,0 +1,50 @@ +import pytest +import re + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, name="log_quries_probability") +node = cluster.add_instance('node', with_zookeeper=False) + +config = ''' + + information + /var/log/clickhouse-server/clickhouse-server.log + +''' + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_log(node): + return node.exec_in_container(["bash", "-c", "cat /var/log/clickhouse-server/clickhouse-server.log"]) + +def test_log_levels_update(start_cluster): + # Make sure that there are enough log messages for the test + for i in range(5): + node.query("SELECT 1") + + log = get_log(node) + assert re.search("(|)", log) + + node.replace_config("/etc/clickhouse-server/config.d/log.xml", config) + node.query("SYSTEM RELOAD CONFIG;") + node.exec_in_container(["bash", "-c", "> /var/log/clickhouse-server/clickhouse-server.log"]) + + for i in range(5): + node.query("SELECT 1") + + log = get_log(node) + assert len(log) > 0 + assert not re.search("(|)", log) + + + From 2f6d771d94a764f57b58140de5cc0bcb709f321f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 14 Oct 2021 18:15:30 +0300 Subject: [PATCH 31/89] Remove trailing whitespace --- src/Common/ProgressIndication.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 1f8fc949886..bf3397f50e1 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -126,7 +126,7 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const { return memory + data.second.memory_usage; }); - return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } From aa56fd15925f53fc0b9ca5b6b0d0604a492b6172 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 21:29:02 +0300 Subject: [PATCH 32/89] Fix style. --- src/TableFunctions/ITableFunctionFileLike.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 49461fe8f46..699ad698bd8 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -16,8 +16,6 @@ #include - - namespace DB { From 700504e7d6c24f268ab97ac8055db1dcf6228da6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 22:09:44 +0300 Subject: [PATCH 33/89] Update Internals.cpp --- programs/copier/Internals.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 84283777c8f..6fc69361c90 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -74,7 +74,6 @@ Block getBlockWithAllStreamData(QueryPipeline pipeline) return block; } - bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) { const auto & storage = storage_ast->as(); From 74e3978110f8d4eb8b06919e6bad3f07017e11f2 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Thu, 14 Oct 2021 22:55:17 +0300 Subject: [PATCH 34/89] commit assert to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index a139bb3cb95..ea9add0d76b 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit a139bb3cb9598c7d92dc69aa6962e3ea4fd18982 +Subproject commit ea9add0d76b0d2ff8616c5e9035389cd159996b7 From bb8105951f651a6dec92c6546d7a4152b5ddb336 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:43:38 +0300 Subject: [PATCH 35/89] Disable fsync_metadata on CI Sometimes we see that pretty simple queries like CREATE DATABASE can take > 30 seconds, let's try to disable fsync. CI: https://clickhouse-test-reports.s3.yandex.net/30065/e5bc573250d3d6938937739b05d6f8cf618722db/functional_stateless_tests_(address).html#fail1 CI: https://clickhouse-test-reports.s3.yandex.net/30065/e5bc573250d3d6938937739b05d6f8cf618722db/functional_stateless_tests_(release).html#fail1 --- tests/config/install.sh | 1 + tests/config/users.d/no_fsync_metadata.xml | 11 +++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/config/users.d/no_fsync_metadata.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index 94ad55504a8..ba6ba0cd07c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -48,6 +48,7 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/no_fsync_metadata.xml b/tests/config/users.d/no_fsync_metadata.xml new file mode 100644 index 00000000000..6b9d69927f8 --- /dev/null +++ b/tests/config/users.d/no_fsync_metadata.xml @@ -0,0 +1,11 @@ + + + + + false + + + From 4ab6f7d771a453eb4f098d5a666b8170eb565f76 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Oct 2021 10:39:31 +0300 Subject: [PATCH 36/89] Finally fix test --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index ee3e6b94537..ac20e60b177 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,7 +1,7 @@ SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; -SYSTEM FLUSH LOGS; - -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40;' and current_database = currentDatabase() and type = 'QueryFinish'; From 03a7f24fa3961f699732ecd1e81455f5ce370689 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 11:14:15 +0300 Subject: [PATCH 37/89] Fix build. --- programs/odbc-bridge/MainHandler.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 6a2e967d179..f3e28891344 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include From ad8a344b46985fe75671e032ccb78982d266a9f0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 13:11:57 +0300 Subject: [PATCH 38/89] Move TTL streams and algo --- src/CMakeLists.txt | 1 + .../TTL}/ITTLAlgorithm.cpp | 2 +- .../TTL}/ITTLAlgorithm.h | 0 .../TTL}/TTLAggregationAlgorithm.cpp | 2 +- .../TTL}/TTLAggregationAlgorithm.h | 2 +- .../TTL}/TTLColumnAlgorithm.cpp | 2 +- .../TTL}/TTLColumnAlgorithm.h | 2 +- .../TTL}/TTLDeleteAlgorithm.cpp | 2 +- .../TTL}/TTLDeleteAlgorithm.h | 2 +- .../TTL}/TTLUpdateInfoAlgorithm.cpp | 2 +- .../TTL}/TTLUpdateInfoAlgorithm.h | 2 +- .../Transforms/TTLCalcTransform.cpp} | 4 +-- .../Transforms/TTLCalcTransform.h} | 2 +- .../Transforms/TTLTransform.cpp} | 10 +++--- .../Transforms/TTLTransform.h} | 4 +-- src/Storages/MergeTree/MergeTask.cpp | 34 +++++++++---------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +-- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 +-- 19 files changed, 42 insertions(+), 41 deletions(-) rename src/{DataStreams => Processors/TTL}/ITTLAlgorithm.cpp (98%) rename src/{DataStreams => Processors/TTL}/ITTLAlgorithm.h (100%) rename src/{DataStreams => Processors/TTL}/TTLAggregationAlgorithm.cpp (99%) rename src/{DataStreams => Processors/TTL}/TTLAggregationAlgorithm.h (96%) rename src/{DataStreams => Processors/TTL}/TTLColumnAlgorithm.cpp (98%) rename src/{DataStreams => Processors/TTL}/TTLColumnAlgorithm.h (95%) rename src/{DataStreams => Processors/TTL}/TTLDeleteAlgorithm.cpp (97%) rename src/{DataStreams => Processors/TTL}/TTLDeleteAlgorithm.h (92%) rename src/{DataStreams => Processors/TTL}/TTLUpdateInfoAlgorithm.cpp (97%) rename src/{DataStreams => Processors/TTL}/TTLUpdateInfoAlgorithm.h (94%) rename src/{DataStreams/TTLCalcInputStream.cpp => Processors/Transforms/TTLCalcTransform.cpp} (97%) rename src/{DataStreams/TTLCalcInputStream.h => Processors/Transforms/TTLCalcTransform.h} (96%) rename src/{DataStreams/TTLBlockInputStream.cpp => Processors/Transforms/TTLTransform.cpp} (95%) rename src/{DataStreams/TTLBlockInputStream.h => Processors/Transforms/TTLTransform.h} (92%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..08755542ed1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -214,6 +214,7 @@ add_object_library(clickhouse_processors_transforms Processors/Transforms) add_object_library(clickhouse_processors_sources Processors/Sources) add_object_library(clickhouse_processors_sinks Processors/Sinks) add_object_library(clickhouse_processors_merges Processors/Merges) +add_object_library(clickhouse_processors_ttl Processors/TTL) add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations) diff --git a/src/DataStreams/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp similarity index 98% rename from src/DataStreams/ITTLAlgorithm.cpp rename to src/Processors/TTL/ITTLAlgorithm.cpp index 7513e0c6ce0..489941950b5 100644 --- a/src/DataStreams/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h similarity index 100% rename from src/DataStreams/ITTLAlgorithm.h rename to src/Processors/TTL/ITTLAlgorithm.h diff --git a/src/DataStreams/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp similarity index 99% rename from src/DataStreams/TTLAggregationAlgorithm.cpp rename to src/Processors/TTL/TTLAggregationAlgorithm.cpp index d012464ea14..5581892f16f 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/Processors/TTL/TTLAggregationAlgorithm.h similarity index 96% rename from src/DataStreams/TTLAggregationAlgorithm.h rename to src/Processors/TTL/TTLAggregationAlgorithm.h index c2f40bab6b9..0e4bf092ed6 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.h +++ b/src/Processors/TTL/TTLAggregationAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/Processors/TTL/TTLColumnAlgorithm.cpp similarity index 98% rename from src/DataStreams/TTLColumnAlgorithm.cpp rename to src/Processors/TTL/TTLColumnAlgorithm.cpp index 71ad2a4e38f..7cef77c185c 100644 --- a/src/DataStreams/TTLColumnAlgorithm.cpp +++ b/src/Processors/TTL/TTLColumnAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/Processors/TTL/TTLColumnAlgorithm.h similarity index 95% rename from src/DataStreams/TTLColumnAlgorithm.h rename to src/Processors/TTL/TTLColumnAlgorithm.h index ddf963eaee2..30de77dcc2a 100644 --- a/src/DataStreams/TTLColumnAlgorithm.h +++ b/src/Processors/TTL/TTLColumnAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLDeleteAlgorithm.cpp b/src/Processors/TTL/TTLDeleteAlgorithm.cpp similarity index 97% rename from src/DataStreams/TTLDeleteAlgorithm.cpp rename to src/Processors/TTL/TTLDeleteAlgorithm.cpp index ea7a0b235ec..eec6b21f949 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.cpp +++ b/src/Processors/TTL/TTLDeleteAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLDeleteAlgorithm.h b/src/Processors/TTL/TTLDeleteAlgorithm.h similarity index 92% rename from src/DataStreams/TTLDeleteAlgorithm.h rename to src/Processors/TTL/TTLDeleteAlgorithm.h index 8ab3f8b63e8..292a29bfa27 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.h +++ b/src/Processors/TTL/TTLDeleteAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp similarity index 97% rename from src/DataStreams/TTLUpdateInfoAlgorithm.cpp rename to src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index 6a983d052c1..eba364aa2b8 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h similarity index 94% rename from src/DataStreams/TTLUpdateInfoAlgorithm.h rename to src/Processors/TTL/TTLUpdateInfoAlgorithm.h index 551211fc47f..45eecbde3d0 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/Processors/Transforms/TTLCalcTransform.cpp similarity index 97% rename from src/DataStreams/TTLCalcInputStream.cpp rename to src/Processors/Transforms/TTLCalcTransform.cpp index fe85e40c003..31fb61239ef 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/Processors/Transforms/TTLCalcTransform.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include namespace DB { diff --git a/src/DataStreams/TTLCalcInputStream.h b/src/Processors/Transforms/TTLCalcTransform.h similarity index 96% rename from src/DataStreams/TTLCalcInputStream.h rename to src/Processors/Transforms/TTLCalcTransform.h index b6318026b8c..14592c07155 100644 --- a/src/DataStreams/TTLCalcInputStream.h +++ b/src/Processors/Transforms/TTLCalcTransform.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/Processors/Transforms/TTLTransform.cpp similarity index 95% rename from src/DataStreams/TTLBlockInputStream.cpp rename to src/Processors/Transforms/TTLTransform.cpp index b476f689e60..a515a50fafb 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,10 +8,10 @@ #include #include -#include -#include -#include -#include +#include +#include +#include +#include namespace DB { diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/Processors/Transforms/TTLTransform.h similarity index 92% rename from src/DataStreams/TTLBlockInputStream.h rename to src/Processors/Transforms/TTLTransform.h index 50b28e81bdf..9207c68448b 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e17a3dcf544..7cf52058558 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1,26 +1,26 @@ -#include "Storages/MergeTree/MergeTask.h" +#include #include #include #include -#include "Common/ActionBlocker.h" +#include -#include "Storages/MergeTree/MergeTreeData.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include "Storages/MergeTree/MergeTreeSequentialSource.h" -#include "Storages/MergeTree/FutureMergedMutatedPart.h" -#include "Processors/Transforms/ExpressionTransform.h" -#include "Processors/Transforms/MaterializingTransform.h" -#include "Processors/Merges/MergingSortedTransform.h" -#include "Processors/Merges/CollapsingSortedTransform.h" -#include "Processors/Merges/SummingSortedTransform.h" -#include "Processors/Merges/ReplacingSortedTransform.h" -#include "Processors/Merges/GraphiteRollupSortedTransform.h" -#include "Processors/Merges/AggregatingSortedTransform.h" -#include "Processors/Merges/VersionedCollapsingTransform.h" -#include "DataStreams/TTLBlockInputStream.h" -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 1ee8423b0d1..5a199610a68 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -15,8 +15,8 @@ #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 50ab26ec470..812299828d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6ad29d01ca6..da79b917b5a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include #include From 65aec857ec89897a98693ccbf0d65a8e311b6b16 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 13:12:59 +0300 Subject: [PATCH 39/89] FunctionsJSON updated --- src/Functions/FunctionsJSON.cpp | 9 ++++++++- src/Functions/FunctionsJSON.h | 17 ----------------- 2 files changed, 8 insertions(+), 18 deletions(-) delete mode 100644 src/Functions/FunctionsJSON.h diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 6889a0d44b9..4e2c42ea3cb 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -58,6 +59,12 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +struct HasIndexOperator : std::false_type {}; + +template +struct HasIndexOperator()[0])>> : std::true_type {}; + /// Functions to parse JSONs and extract values from it. /// The first argument of all these functions gets a JSON, @@ -279,7 +286,7 @@ private: return true; } - if constexpr (FunctionJSONHelpersDetails::has_index_operator::value) + if constexpr (HasIndexOperator::value) { if (element.isObject()) { diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h deleted file mode 100644 index 4ef43eb637f..00000000000 --- a/src/Functions/FunctionsJSON.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -namespace FunctionJSONHelpersDetails -{ - template - struct has_index_operator : std::false_type {}; - - template - struct has_index_operator()[0])>> : std::true_type {}; -} - -} From 41dc195b345c199e4fef4d62d57e9d1cab3cf69c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 13:15:14 +0300 Subject: [PATCH 40/89] Fix build. --- programs/odbc-bridge/MainHandler.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index f3e28891344..6362c7dfa9b 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include From a744097fb65fbfa22809c5c640f4ff73f07fd2ba Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 15:34:13 +0300 Subject: [PATCH 41/89] Fixed build --- src/Functions/FunctionsJSON.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 4e2c42ea3cb..9558b856511 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include From f69daace7b400735dec36e9190add9fd9e7651c9 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Fri, 15 Oct 2021 16:01:28 +0300 Subject: [PATCH 42/89] attemp to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index ea9add0d76b..c1e01a441d6 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit ea9add0d76b0d2ff8616c5e9035389cd159996b7 +Subproject commit c1e01a441d6db234f4f12e63a7657d1f9e6db9c1 From e9bf496d42435c02a96518adf33e19dba4c4f004 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 14:35:50 +0000 Subject: [PATCH 43/89] Added an ability to generate data in build time --- CMakeLists.txt | 15 ++++++ .../libprotobuf-mutator-cmake/CMakeLists.txt | 7 +-- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 48 +++++++++++++++++-- .../codegen_fuzzer/codegen_select_fuzzer.cpp | 1 - src/Parsers/fuzzers/codegen_fuzzer/gen.py | 19 ++++---- 5 files changed, 70 insertions(+), 20 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f32841c15da..c2d3421b28e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -136,6 +136,21 @@ if (ENABLE_FUZZING) message (STATUS "Fuzzing instrumentation enabled") set (FUZZER "libfuzzer") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -nostdlib++") + set (ENABLE_CLICKHOUSE_ODBC_BRIDGE OFF) + set (ENABLE_LIBRARIES 0) + set (ENABLE_SSL 1) + set (USE_INTERNAL_SSL_LIBRARY 1) + set (USE_UNWIND ON) + set (ENABLE_EMBEDDED_COMPILER 0) + set (ENABLE_EXAMPLES 0) + set (ENABLE_UTILS 0) + set (ENABLE_THINLTO 0) + set (ENABLE_TCMALLOC 0) + set (ENABLE_JEMALLOC 0) + set (ENABLE_CHECK_HEAVY_BUILDS 1) + set (GLIBC_COMPATIBILITY OFF) + set (ENABLE_PROTOBUF ON) + set (USE_INTERNAL_PROTOBUF_LIBRARY ON) endif() # Global libraries diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt index 93eafc85b7d..978b1e732ba 100644 --- a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -1,5 +1,4 @@ set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator) -set(NO_FUZZING_FLAGS "-fno-sanitize=fuzzer -fsanitize-coverage=0") add_library(protobuf-mutator ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_macro.cc @@ -10,8 +9,6 @@ add_library(protobuf-mutator ${LIBRARY_DIR}/src/utf8_fix.cc) target_include_directories(protobuf-mutator BEFORE PRIVATE "${LIBRARY_DIR}") -# target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") +target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") -target_link_libraries(protobuf-mutator ${PROTOBUF_LIBRARIES}) -set_target_properties(protobuf-mutator PROPERTIES - COMPILE_FLAGS "${NO_FUZZING_FLAGS}") +target_link_libraries(protobuf-mutator ${Protobuf_LIBRARY}) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 370396a5e8e..3c274bd4786 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -1,7 +1,47 @@ find_package(Protobuf REQUIRED) -protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS "out.proto") -set(FUZZER_SRCS codegen_select_fuzzer.cpp out.cpp ${PROTO_SRCS} ${PROTO_HDRS}) +set (CURRENT_DIR_IN_SOURCES "${ClickHouse_SOURCE_DIR}/src/Parsers/fuzzers/codegen_fuzzer") +set (CURRENT_DIR_IN_BINARY "${ClickHouse_BINARY_DIR}/src/Parsers/fuzzers/codegen_fuzzer") + +# Copy scripts and template file to build directory to generate .proto and .cpp file from them +configure_file( + "${CURRENT_DIR_IN_SOURCES}/gen.py" + "${CURRENT_DIR_IN_BINARY}/gen.py" + COPYONLY) +configure_file( + "${CURRENT_DIR_IN_SOURCES}/update.sh" + "${CURRENT_DIR_IN_BINARY}/update.sh" + COPYONLY) + +# Delete this and uncomment the next block if you want to generage clickhouse.g automatically +configure_file( + "${CURRENT_DIR_IN_SOURCES}/clickhouse.g" + "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + COPYONLY) + +# # Uncomment to generate clickhouse.g automatically +# configure_file( +# "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" +# "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" +# COPYONLY) + +# # Note that it depends on all.dict file! +# add_custom_command( +# OUTPUT +# "${CURRENT_DIR_IN_BINARY}/clickhouse.g" +# COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" +# ) + +add_custom_command( + OUTPUT + "${CURRENT_DIR_IN_BINARY}/out.cpp" + "${CURRENT_DIR_IN_BINARY}/out.proto" + COMMAND python3 gen.py clickhouse.g out.cpp out.proto + # DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" +) + +PROTOBUF_GENERATE_CPP(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") +set(FUZZER_SRCS codegen_select_fuzzer.cpp "${CURRENT_DIR_IN_BINARY}/out.cpp" ${PROTO_SRCS} ${PROTO_HDRS}) set(CMAKE_INCLUDE_CURRENT_DIR TRUE) @@ -9,7 +49,7 @@ add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") -target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIR}" "${CMAKE_CURRENT_BINARY_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") -target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) +target_link_libraries(codegen_select_fuzzer PRIVATE protobuf-mutator dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index a68fac4f585..418a5014657 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -11,7 +11,6 @@ #include "out.pb.h" - void GenerateSentence(const Sentence&, std::string &, int); diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py index e96bc6ae9f6..b594d1026e3 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/gen.py +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -58,14 +58,14 @@ class Parser: self.var_id = -1 self.cur_tok = None self.includes = [] - + self.proto = '' self.cpp = '' def parse_file(self, filename): with open(filename) as f: self.text = f.read() - + while self.parse_statement() is not None: pass @@ -90,12 +90,12 @@ class Parser: def parse_var_value(self): i = self.text.find(' ') - + id_, self.text = self.text[1:i], self.text[i+1:] self.var_id = int(id_) self.cur_tok = TOKEN_VAR return TOKEN_VAR - + def parse_txt_value(self): if self.text[0] != '"': raise Exception("parse_txt_value: expected quote at the start") @@ -116,7 +116,7 @@ class Parser: else: c, self.text = self.text[0], self.text[1:] self.t += c - + self.text = self.text[1:] self.cur_tok = TOKEN_TEXT return TOKEN_TEXT @@ -137,7 +137,7 @@ class Parser: index = self.text.find('\n') self.text = self.text[index:] - + def parse_statement(self): if self.skip_ws() is None: return None @@ -146,7 +146,7 @@ class Parser: if self.cur_tok == TOKEN_SLASH: self.skip_line() return TOKEN_SLASH - + chain = [] while self.cur_tok != TOKEN_SEMI: if self.cur_tok == TOKEN_TEXT: @@ -164,7 +164,7 @@ class Parser: def generate(self): self.proto = 'syntax = "proto3";\n\n' - self.cpp = '#include \n#include \n#include \n\n#include \n\n' + self.cpp = '#include \n#include \n#include \n\n#include \n\n' for incl_file in self.includes: self.cpp += f'#include "{incl_file}"\n' @@ -228,7 +228,7 @@ def main(args): p = Parser() p.add_include(include_filename) p.parse_file(input_file) - + cpp, proto = p.generate() proto = proto.replace('\t', ' ' * 4) @@ -246,4 +246,3 @@ if __name__ == '__main__': print(f"Usage {sys.argv[0]} ") sys.exit(1) main(sys.argv[1:]) - From 058081bcb2810082bf14eea36402860653169b44 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Fri, 15 Oct 2021 18:18:02 +0300 Subject: [PATCH 44/89] fixed typo in gen.py; clickhouse.g applying Nikita's in build-time generation --- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 23 +- .../fuzzers/codegen_fuzzer/clickhouse.g | 524 ------------------ src/Parsers/fuzzers/codegen_fuzzer/gen.py | 2 +- src/Parsers/fuzzers/codegen_fuzzer/update.sh | 4 +- 4 files changed, 11 insertions(+), 542 deletions(-) delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 3c274bd4786..f87340b5024 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -13,24 +13,17 @@ configure_file( "${CURRENT_DIR_IN_BINARY}/update.sh" COPYONLY) -# Delete this and uncomment the next block if you want to generage clickhouse.g automatically configure_file( - "${CURRENT_DIR_IN_SOURCES}/clickhouse.g" - "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" + "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" COPYONLY) -# # Uncomment to generate clickhouse.g automatically -# configure_file( -# "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" -# "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" -# COPYONLY) - -# # Note that it depends on all.dict file! -# add_custom_command( -# OUTPUT -# "${CURRENT_DIR_IN_BINARY}/clickhouse.g" -# COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" -# ) +# Note that it depends on all.dict file! +add_custom_command( + OUTPUT + "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" +) add_custom_command( OUTPUT diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g deleted file mode 100644 index edd5acf513d..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g +++ /dev/null @@ -1,524 +0,0 @@ -" "; -" "; -" "; -";"; - - -"(" $1 ")"; -"(" $1 ", " $2 ")"; -"(" $1 ", " $2 ", " $3 ")"; - -$1 ", " $2 ; -$1 ", " $2 ", " $3 ; -$1 ", " $2 ", " $3 ", " $4 ; -$1 ", " $2 ", " $3 ", " $4 ", " $5 ; - -"[" $1 ", " $2 "]"; -"[" $1 ", " $2 ", " $3 "]"; -"[" $1 ", " $2 ", " $3 ", " $4 "]"; -"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; - -$0 "(" $1 ")"; -$0 "(" $1 ", " $2 ")"; -$0 "(" $1 ", " $2 ", " $3 ")"; - -$1 " as " $2 ; - - -// TODO: add more clickhouse specific stuff -"SELECT " $1 " FROM " $2 " WHERE " $3 ; -"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; -"SELECT " $1 " FROM " $2 " SORT BY " $3 ; -"SELECT " $1 " FROM " $2 " LIMIT " $3 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 ; -"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; -"SELECT " $1 " INTO OUTFILE " $2 ; - -"WITH " $1 " AS " $2 ; - -"{" $1 ":" $2 "}"; -"[" $1 "," $2 "]"; -"[]"; - - -" x "; -"x"; -" `x` "; -"`x`"; - -" \"value\" "; -"\"value\""; -" 0 "; -"0"; -"1"; -"2"; -"123123123123123123"; -"182374019873401982734091873420923123123123123123"; -"1e-1"; -"1.1"; -"\"\""; -" '../../../../../../../../../etc/passwd' "; - -"/"; -"="; -"=="; -"!="; -"<>"; -"<"; -"<="; -">"; -">="; -"<<"; -"|<<"; -"&"; -"|"; -"||"; -"<|"; -"|>"; -"+"; -"-"; -"~"; -"*"; -"/"; -"\\"; -"%"; -""; -"."; -","; -","; -","; -","; -","; -","; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"?"; -":"; -"@"; -"@@"; -"$"; -"\""; -"`"; -"{"; -"}"; -"^"; -"::"; -"->"; -"]"; -"["; - -" ADD "; -" ADD COLUMN "; -" ADD CONSTRAINT "; -" ADD INDEX "; -" AFTER "; -" AggregateFunction "; -" aggThrow "; -" ALL "; -" ALTER LIVE VIEW "; -" ALTER TABLE "; -" and "; -" ANTI "; -" any "; -" anyHeavy "; -" anyLast "; -" argMax "; -" argMin "; -" array "; -" Array "; -" arrayAll "; -" arrayAUC "; -" arrayCompact "; -" arrayConcat "; -" arrayCount "; -" arrayCumSum "; -" arrayCumSumNonNegative "; -" arrayDifference "; -" arrayDistinct "; -" arrayElement "; -" arrayEnumerate "; -" arrayEnumerateDense "; -" arrayEnumerateDenseRanked "; -" arrayEnumerateUniq "; -" arrayEnumerateUniqRanked "; -" arrayExists "; -" arrayFill "; -" arrayFilter "; -" arrayFirst "; -" arrayFirstIndex "; -" arrayFlatten "; -" arrayIntersect "; -" arrayJoin "; -" ARRAY JOIN "; -" arrayMap "; -" arrayPopBack "; -" arrayPopFront "; -" arrayPushBack "; -" arrayPushFront "; -" arrayReduce "; -" arrayReduceInRanges "; -" arrayResize "; -" arrayReverse "; -" arrayReverseFill "; -" arrayReverseSort "; -" arrayReverseSplit "; -" arraySlice "; -" arraySort "; -" arraySplit "; -" arraySum "; -" arrayUniq "; -" arrayWithConstant "; -" arrayZip "; -" AS "; -" ASC "; -" ASCENDING "; -" ASOF "; -" AST "; -" ATTACH "; -" ATTACH PART "; -" ATTACH PARTITION "; -" avg "; -" avgWeighted "; -" basename "; -" BETWEEN "; -" BOTH "; -" boundingRatio "; -" BY "; -" CAST "; -" categoricalInformationValue "; -" CHECK "; -" CHECK TABLE "; -" CLEAR COLUMN "; -" CLEAR INDEX "; -" COLLATE "; -" COLUMNS "; -" COMMENT COLUMN "; -" CONSTRAINT "; -" corr "; -" corrStable "; -" count "; -" countEqual "; -" covarPop "; -" covarPopStable "; -" covarSamp "; -" covarSampStable "; -" CREATE "; -" CROSS "; -" CUBE "; -" cutFragment "; -" cutQueryString "; -" cutQueryStringAndFragment "; -" cutToFirstSignificantSubdomain "; -" cutURLParameter "; -" cutWWW "; -" D "; -" DATABASE "; -" DATABASES "; -" Date "; -" DATE "; -" DATE_ADD "; -" DATEADD "; -" DATE_DIFF "; -" DATEDIFF "; -" DATE_SUB "; -" DATESUB "; -" DateTime "; -" DateTime64 "; -" DAY "; -" DD "; -" Decimal "; -" Decimal128 "; -" Decimal32 "; -" Decimal64 "; -" decodeURLComponent "; -" DEDUPLICATE "; -" DELETE "; -" DELETE WHERE "; -" DESC "; -" DESCENDING "; -" DESCRIBE "; -" DETACH "; -" DETACH PARTITION "; -" DICTIONARIES "; -" DICTIONARY "; -" DISTINCT "; -" domain "; -" domainWithoutWWW "; -" DROP "; -" DROP COLUMN "; -" DROP CONSTRAINT "; -" DROP DETACHED PART "; -" DROP DETACHED PARTITION "; -" DROP INDEX "; -" DROP PARTITION "; -" emptyArrayToSingle "; -" ENGINE "; -" entropy "; -" Enum "; -" Enum16 "; -" Enum8 "; -" EVENTS "; -" EXCHANGE TABLES "; -" EXISTS "; -" EXTRACT "; -" extractURLParameter "; -" extractURLParameterNames "; -" extractURLParameters "; -" FETCH PARTITION "; -" FETCH PART "; -" FINAL "; -" FIRST "; -" firstSignificantSubdomain "; -" FixedString "; -" Float32 "; -" Float64 "; -" FOR "; -" ForEach "; -" FORMAT "; -" fragment "; -" FREEZE "; -" FROM "; -" FULL "; -" FUNCTION "; -" __getScalar "; -" GLOBAL "; -" GRANULARITY "; -" groupArray "; -" groupArrayInsertAt "; -" groupArrayMovingAvg "; -" groupArrayMovingSum "; -" groupArraySample "; -" groupBitAnd "; -" groupBitmap "; -" groupBitmapAnd "; -" groupBitmapOr "; -" groupBitmapXor "; -" groupBitOr "; -" groupBitXor "; -" GROUP BY "; -" groupUniqArray "; -" has "; -" hasAll "; -" hasAny "; -" HAVING "; -" HH "; -" histogram "; -" HOUR "; -" ID "; -" if "; -" IF EXISTS "; -" IF NOT EXISTS "; -" IN "; -" INDEX "; -" indexOf "; -" INNER "; -" IN PARTITION "; -" INSERT INTO "; -" Int16 "; -" Int32 "; -" Int64 "; -" Int8 "; -" INTERVAL "; -" IntervalDay "; -" IntervalHour "; -" IntervalMinute "; -" IntervalMonth "; -" IntervalQuarter "; -" IntervalSecond "; -" IntervalWeek "; -" IntervalYear "; -" INTO OUTFILE "; -" JOIN "; -" kurtPop "; -" kurtSamp "; -" LAST "; -" LAYOUT "; -" LEADING "; -" LEFT "; -" LEFT ARRAY JOIN "; -" length "; -" LIFETIME "; -" LIKE "; -" LIMIT "; -" LIVE "; -" LOCAL "; -" LowCardinality "; -" LTRIM "; -" M "; -" MATERIALIZED "; -" MATERIALIZE INDEX "; -" MATERIALIZE TTL "; -" max "; -" maxIntersections "; -" maxIntersectionsPosition "; -" Merge "; -" MI "; -" min "; -" MINUTE "; -" MM "; -" MODIFY "; -" MODIFY COLUMN "; -" MODIFY ORDER BY "; -" MODIFY QUERY "; -" MODIFY SETTING "; -" MODIFY TTL "; -" MONTH "; -" MOVE PART "; -" MOVE PARTITION "; -" movingXXX "; -" N "; -" NAME "; -" Nested "; -" NO DELAY "; -" NONE "; -" not "; -" nothing "; -" Nothing "; -" Null "; -" Nullable "; -" NULLS "; -" OFFSET "; -" ON "; -" ONLY "; -" OPTIMIZE TABLE "; -" ORDER BY "; -" OR REPLACE "; -" OUTER "; -" PARTITION "; -" PARTITION BY "; -" path "; -" pathFull "; -" POPULATE "; -" PREWHERE "; -" PRIMARY KEY "; -" protocol "; -" Q "; -" QQ "; -" QUARTER "; -" queryString "; -" queryStringAndFragment "; -" range "; -" REFRESH "; -" RENAME COLUMN "; -" RENAME TABLE "; -" REPLACE PARTITION "; -" Resample "; -" RESUME "; -" retention "; -" RIGHT "; -" ROLLUP "; -" RTRIM "; -" S "; -" SAMPLE "; -" SAMPLE BY "; -" SECOND "; -" SELECT "; -" SEMI "; -" sequenceCount "; -" sequenceMatch "; -" SET "; -" SETTINGS "; -" SHOW "; -" SHOW PROCESSLIST "; -" simpleLinearRegression "; -" skewPop "; -" skewSamp "; -" SOURCE "; -" SQL_TSI_DAY "; -" SQL_TSI_HOUR "; -" SQL_TSI_MINUTE "; -" SQL_TSI_MONTH "; -" SQL_TSI_QUARTER "; -" SQL_TSI_SECOND "; -" SQL_TSI_WEEK "; -" SQL_TSI_YEAR "; -" SS "; -" State "; -" stddevPop "; -" stddevPopStable "; -" stddevSamp "; -" stddevSampStable "; -" STEP "; -" stochasticLinearRegression "; -" stochasticLogisticRegression "; -" String "; -" SUBSTRING "; -" sum "; -" sumKahan "; -" sumMap "; -" sumMapFiltered "; -" sumMapFilteredWithOverflow "; -" sumMapWithOverflow "; -" sumWithOverflow "; -" SUSPEND "; -" TABLE "; -" TABLES "; -" TEMPORARY "; -" TIMESTAMP "; -" TIMESTAMP_ADD "; -" TIMESTAMPADD "; -" TIMESTAMP_DIFF "; -" TIMESTAMPDIFF "; -" TIMESTAMP_SUB "; -" TIMESTAMPSUB "; -" TO "; -" TO DISK "; -" TOP "; -" topK "; -" topKWeighted "; -" topLevelDomain "; -" TO TABLE "; -" TOTALS "; -" TO VOLUME "; -" TRAILING "; -" TRIM "; -" TRUNCATE "; -" TTL "; -" Tuple "; -" TYPE "; -" UInt16 "; -" UInt32 "; -" UInt64 "; -" UInt8 "; -" uniq "; -" uniqCombined "; -" uniqCombined64 "; -" uniqExact "; -" uniqHLL12 "; -" uniqUpTo "; -" UPDATE "; -" URLHierarchy "; -" URLPathHierarchy "; -" USE "; -" USING "; -" UUID "; -" VALUES "; -" varPop "; -" varPopStable "; -" varSamp "; -" varSampStable "; -" VIEW "; -" WATCH "; -" WEEK "; -" WHERE "; -" windowFunnel "; -" WITH "; -" WITH FILL "; -" WITH TIES "; -" WK "; -" WW "; -" YEAR "; -" YY "; -" YYYY "; diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py index b594d1026e3..95936247489 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/gen.py +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -154,7 +154,7 @@ class Parser: elif self.cur_tok == TOKEN_VAR: chain.append(Var(self.var_id)) else: - self.fatal_parsing_error(f"unexpected token {self.tok}") + self.fatal_parsing_error(f"unexpected token {self.cur_tok}") self.get_next_token() if not chain: diff --git a/src/Parsers/fuzzers/codegen_fuzzer/update.sh b/src/Parsers/fuzzers/codegen_fuzzer/update.sh index 0982d6d0686..daee56dcea1 100755 --- a/src/Parsers/fuzzers/codegen_fuzzer/update.sh +++ b/src/Parsers/fuzzers/codegen_fuzzer/update.sh @@ -20,11 +20,11 @@ _main() { while read line; do [[ -z "$line" ]] && continue - echo $line | sed -e 's/"\(.*\)"/" \1 ";/g' + echo $line | sed -e '/^#/d' -e 's/"\(.*\)"/" \1 ";/g' done < $dict_filename >> clickhouse.g } _main "$@" -# Sample run: ./update.sh ../../../../tests/fuzz/ast.dict +# Sample run: ./update.sh ${CLICKHOUSE_SOURCE_DIR}/tests/fuzz/all.dict # then run `python ./gen.py clickhouse.g out.cpp out.proto` to generate new files with tokens. Rebuild fuzzer From 8dde0a5dfbc457af33ea9c9b4ce6f15465e62185 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:35:20 +0300 Subject: [PATCH 45/89] ComplexKeyHashedDictionary fix keys copy --- src/Dictionaries/HashedDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1f3821096da..eec7cec0285 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -667,7 +667,7 @@ Pipe HashedDictionary::read(const Names & column_na if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template From c9638bacbf625fd8c08667175e42428add6fbc1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:53:17 +0300 Subject: [PATCH 46/89] Added HashedArray dictionary --- src/Dictionaries/CMakeLists.txt | 1 + src/Dictionaries/HashedArrayDictionary.cpp | 689 ++++++++++++++++++ src/Dictionaries/HashedArrayDictionary.h | 211 ++++++ src/Dictionaries/registerDictionaries.cpp | 2 + tests/performance/hashed_array_dictionary.xml | 126 ++++ ...shed_array_dictionary_simple_key.reference | 66 ++ ...098_hashed_array_dictionary_simple_key.sql | 125 ++++ ...hed_array_dictionary_complex_key.reference | 56 ++ 8 files changed, 1276 insertions(+) create mode 100644 src/Dictionaries/HashedArrayDictionary.cpp create mode 100644 src/Dictionaries/HashedArrayDictionary.h create mode 100644 tests/performance/hashed_array_dictionary.xml create mode 100644 tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference create mode 100644 tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql create mode 100644 tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index bc5f0dc9567..b1b3d6d55e0 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -10,6 +10,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW set_source_files_properties( FlatDictionary.cpp HashedDictionary.cpp + HashedArrayDictionary.cpp CacheDictionary.cpp RangeHashedDictionary.cpp DirectDictionary.cpp diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp new file mode 100644 index 00000000000..7160cb67e10 --- /dev/null +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -0,0 +1,689 @@ +#include "HashedArrayDictionary.h" + +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int DICTIONARY_IS_EMPTY; + extern const int UNSUPPORTED_METHOD; +} + +template +HashedArrayDictionary::HashedArrayDictionary( + const StorageID & dict_id_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const HashedArrayDictionaryStorageConfiguration & configuration_, + BlockPtr update_field_loaded_block_) + : IDictionary(dict_id_) + , dict_struct(dict_struct_) + , source_ptr(std::move(source_ptr_)) + , configuration(configuration_) + , update_field_loaded_block(std::move(update_field_loaded_block_)) +{ + createAttributes(); + loadData(); + calculateBytesAllocated(); +} + +template +ColumnPtr HashedArrayDictionary::getColumn( + const std::string & attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types [[maybe_unused]], + const ColumnPtr & default_values_column) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + dict_struct.validateKeyTypes(key_types); + + ColumnPtr result; + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + + const size_t size = extractor.getKeysSize(); + + const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + auto & attribute = attributes[attribute_index]; + + bool is_attribute_nullable = attribute.is_index_null.has_value(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (attribute.is_index_null) + { + col_null_map_to = ColumnUInt8::create(size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnProvider = DictionaryAttributeColumnProvider; + + DictionaryDefaultValueExtractor default_value_extractor(dictionary_attribute.null_value, default_values_column); + + auto column = ColumnProvider::getColumn(dictionary_attribute, size); + + if constexpr (std::is_same_v) + { + auto * out = column.get(); + + getItemsImpl( + attribute, + extractor, + [&](const size_t, const Array & value, bool) { out->insert(value); }, + default_value_extractor); + } + else if constexpr (std::is_same_v) + { + auto * out = column.get(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + extractor, + [&](size_t row, const StringRef value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out->insertData(value.data, value.size); + }, + default_value_extractor); + else + getItemsImpl( + attribute, + extractor, + [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + default_value_extractor); + } + else + { + auto & out = column->getData(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + extractor, + [&](size_t row, const auto value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out[row] = value; + }, + default_value_extractor); + else + getItemsImpl( + attribute, + extractor, + [&](size_t row, const auto value, bool) { out[row] = value; }, + default_value_extractor); + } + + result = std::move(column); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (is_attribute_nullable) + result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + + return result; +} + +template +ColumnUInt8::Ptr HashedArrayDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + dict_struct.validateKeyTypes(key_types); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + + size_t keys_size = extractor.getKeysSize(); + + auto result = ColumnUInt8::create(keys_size, false); + auto & out = result->getData(); + + if (attributes.empty()) + { + query_count.fetch_add(keys_size, std::memory_order_relaxed); + return result; + } + + size_t keys_found = 0; + + for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) + { + auto requested_key = extractor.extractCurrentKey(); + + out[requested_key_index] = key_attribute.container.find(requested_key) != key_attribute.container.end(); + + keys_found += out[requested_key_index]; + extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; +} + +template +ColumnPtr HashedArrayDictionary::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const auto & key_attribute_container = key_attribute.container; + + const UInt64 null_value = dictionary_attribute.null_value.template get(); + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) { return key_attribute_container.find(key) != key_attribute_container.end(); }; + + size_t keys_found = 0; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = key_attribute_container.find(hierarchy_key); + + if (it != key_attribute_container.end()) + result = parent_keys_container[it->getMapped()]; + + keys_found += result.has_value(); + + return result; + }; + + auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, null_value, is_key_valid_func, get_parent_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return dictionary_hierarchy_array; + } + else + { + return nullptr; + } +} + +template +ColumnUInt8::Ptr HashedArrayDictionary::isInHierarchy( + ColumnPtr key_column [[maybe_unused]], + ColumnPtr in_key_column [[maybe_unused]], + const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + PaddedPODArray keys_in_backup_storage; + const auto & keys_in = getColumnVectorData(this, in_key_column, keys_in_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const auto & key_attribute_container = key_attribute.container; + + const UInt64 null_value = dictionary_attribute.null_value.template get(); + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) { return key_attribute_container.find(key) != key_attribute_container.end(); }; + + size_t keys_found = 0; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = key_attribute_container.find(hierarchy_key); + + if (it != key_attribute_container.end()) + result = parent_keys_container[it->getMapped()]; + + keys_found += result.has_value(); + + return result; + }; + + auto result = getKeysIsInHierarchyColumn(keys, keys_in, null_value, is_key_valid_func, get_parent_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; + } + else + { + return nullptr; + } +} + +template +ColumnPtr HashedArrayDictionary::getDescendants( + ColumnPtr key_column [[maybe_unused]], + const DataTypePtr &, + size_t level [[maybe_unused]]) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup; + const auto & keys = getColumnVectorData(this, key_column, keys_backup); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + const auto & key_attribute_container = key_attribute.container; + + HashMap index_to_key; + index_to_key.reserve(key_attribute.container.size()); + + for (auto & [key, value] : key_attribute_container) + index_to_key[value] = key; + + HashMap> parent_to_child; + + for (size_t i = 0; i < parent_keys_container.size(); ++i) + { + const auto * it = index_to_key.find(i); + if (it == index_to_key.end()) + continue; + + auto parent_key = it->getMapped(); + auto child_key = parent_keys_container[i]; + parent_to_child[parent_key].emplace_back(child_key); + } + + size_t keys_found = 0; + auto result = getKeysDescendantsArray(keys, parent_to_child, level, keys_found); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; + } + else + { + return nullptr; + } +} + +template +void HashedArrayDictionary::createAttributes() +{ + const auto size = dict_struct.attributes.size(); + attributes.reserve(size); + + for (const auto & dictionary_attribute : dict_struct.attributes) + { + auto type_call = [&, this](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + auto is_index_null = dictionary_attribute.is_nullable ? std::make_optional>() : std::optional>{}; + std::unique_ptr string_arena = std::is_same_v ? std::make_unique() : nullptr; + Attribute attribute{dictionary_attribute.underlying_type, AttributeContainerType(), std::move(is_index_null), std::move(string_arena)}; + attributes.emplace_back(std::move(attribute)); + }; + + callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); + } +} + +template +void HashedArrayDictionary::updateData() +{ + if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) + { + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + /// We are using this to keep saved data if input stream consists of multiple blocks + if (!update_field_loaded_block) + update_field_loaded_block = std::make_shared(block.cloneEmpty()); + + for (size_t attribute_index = 0; attribute_index < block.columns(); ++attribute_index) + { + const IColumn & update_column = *block.getByPosition(attribute_index).column.get(); + MutableColumnPtr saved_column = update_field_loaded_block->getByPosition(attribute_index).column->assumeMutable(); + saved_column->insertRangeFrom(update_column, 0, update_column.size()); + } + } + } + else + { + auto pipe = source_ptr->loadUpdatedAll(); + mergeBlockWithPipe( + dict_struct.getKeysSize(), + *update_field_loaded_block, + std::move(pipe)); + } + + if (update_field_loaded_block) + { + resize(update_field_loaded_block->rows()); + blockToAttributes(*update_field_loaded_block.get()); + } +} + +template +void HashedArrayDictionary::blockToAttributes(const Block & block [[maybe_unused]]) +{ + size_t skip_keys_size_offset = dict_struct.getKeysSize(); + + Columns key_columns; + key_columns.reserve(skip_keys_size_offset); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + key_columns.emplace_back(block.safeGetByPosition(i).column); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + Field column_value_to_insert; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + auto it = key_attribute.container.find(key); + + if (it != key_attribute.container.end()) + { + keys_extractor.rollbackCurrentKey(); + continue; + } + + if constexpr (std::is_same_v) + key = copyKeyInArena(key); + + key_attribute.container.insert({key, element_count}); + + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) + { + const IColumn & attribute_column = *block.safeGetByPosition(skip_keys_size_offset + attribute_index).column; + auto & attribute = attributes[attribute_index]; + bool attribute_is_nullable = attribute.is_index_null.has_value(); + + attribute_column.get(key_index, column_value_to_insert); + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using AttributeValueType = DictionaryValueType; + + auto & attribute_container = std::get>(attribute.container); + attribute_container.emplace_back(); + + if (attribute_is_nullable) { + attribute.is_index_null->emplace_back(); + } + + if (attribute_is_nullable && column_value_to_insert.isNull()) + { + (*attribute.is_index_null).back() = true; + return; + } + + if constexpr (std::is_same_v) + { + String & value_to_insert = column_value_to_insert.get(); + size_t value_to_insert_size = value_to_insert.size(); + + const char * string_in_arena = attribute.string_arena->insert(value_to_insert.data(), value_to_insert_size); + + StringRef string_in_arena_reference = StringRef{string_in_arena, value_to_insert_size}; + attribute_container.back() = string_in_arena_reference; + } + else + { + auto value_to_insert = column_value_to_insert.get>(); + attribute_container.back() = value_to_insert; + } + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + } + + ++element_count; + keys_extractor.rollbackCurrentKey(); + } +} + +template +void HashedArrayDictionary::resize(size_t added_rows) +{ + if (unlikely(!added_rows)) + return; + + key_attribute.container.reserve(added_rows); +} + +template +template +void HashedArrayDictionary::getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value [[maybe_unused]], + DefaultValueExtractor & default_value_extractor) const +{ + const auto & key_attribute_container = key_attribute.container; + const auto & attribute_container = std::get>(attribute.container); + const size_t keys_size = keys_extractor.getKeysSize(); + + size_t keys_found = 0; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + const auto it = key_attribute_container.find(key); + + if (it != key_attribute_container.end()) + { + size_t element_index = it->getMapped(); + + const auto & element = attribute_container[element_index]; + + if constexpr (is_nullable) + set_value(key_index, element, (*attribute.is_index_null)[element_index]); + else + set_value(key_index, element, false); + + ++keys_found; + } + else + { + if constexpr (is_nullable) + set_value(key_index, default_value_extractor[key_index], default_value_extractor.isNullAt(key_index)); + else + set_value(key_index, default_value_extractor[key_index], false); + } + + keys_extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); +} + +template +StringRef HashedArrayDictionary::copyKeyInArena(StringRef key) +{ + size_t key_size = key.size; + char * place_for_key = complex_key_arena.alloc(key_size); + memcpy(reinterpret_cast(place_for_key), reinterpret_cast(key.data), key_size); + StringRef updated_key{place_for_key, key_size}; + return updated_key; +} + +template +void HashedArrayDictionary::loadData() +{ + if (!source_ptr->hasUpdateField()) + { + QueryPipeline pipeline; + pipeline = QueryPipeline(source_ptr->loadAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + resize(block.rows()); + blockToAttributes(block); + } + } + else + { + updateData(); + } + + if (configuration.require_nonempty && 0 == element_count) + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, + "{}: dictionary source is empty and 'require_nonempty' property is set.", + full_name); +} + +template +void HashedArrayDictionary::calculateBytesAllocated() +{ + bytes_allocated += attributes.size() * sizeof(attributes.front()); + + bytes_allocated += key_attribute.container.size(); + + for (auto & attribute : attributes) + { + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + const auto & container = std::get>(attribute.container); + bytes_allocated += sizeof(AttributeContainerType); + + if constexpr (std::is_same_v) + { + /// It is not accurate calculations + bytes_allocated += sizeof(Array) * container.size(); + } + else + { + bytes_allocated += container.allocated_bytes(); + } + + bucket_count = container.capacity(); + + if constexpr (std::is_same_v) + bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (attribute.string_arena) + bytes_allocated += attribute.string_arena->size(); + } + + bytes_allocated += complex_key_arena.size(); + + if (update_field_loaded_block) + bytes_allocated += update_field_loaded_block->allocatedBytes(); +} + +template +Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size) const +{ + PaddedPODArray keys; + + for (auto & [key, value] : key_attribute.container) + keys.emplace_back(key); + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + else + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); +} + +template class HashedArrayDictionary; +template class HashedArrayDictionary; + +void registerDictionaryArrayHashed(DictionaryFactory & factory) +{ + auto create_layout = [](const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr, + DictionaryKeyType dictionary_key_type) -> DictionaryPtr + { + if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed array dictionary"); + else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed array dictionary"); + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{}: elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + full_name); + + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + + HashedArrayDictionaryStorageConfiguration configuration{require_nonempty, dict_lifetime}; + + if (dictionary_key_type == DictionaryKeyType::Simple) + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + else + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + }; + + using namespace std::placeholders; + + factory.registerLayout("hashed_array", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple); }, false); + factory.registerLayout("complex_key_hashed_array", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex); }, true); +} + +} diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h new file mode 100644 index 00000000000..053813bdc44 --- /dev/null +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -0,0 +1,211 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +/** This dictionary stores all attributes in arrays. + * Key is stored in hash table and value is index into attribute array. + */ + +namespace DB +{ + +struct HashedArrayDictionaryStorageConfiguration +{ + const bool require_nonempty; + const DictionaryLifetime lifetime; +}; + +template +class HashedArrayDictionary final : public IDictionary +{ +public: + using KeyType = std::conditional_t; + + HashedArrayDictionary( + const StorageID & dict_id_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const HashedArrayDictionaryStorageConfiguration & configuration_, + BlockPtr update_field_loaded_block_ = nullptr); + + std::string getTypeName() const override + { + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + return "HashedArray"; + else + return "ComplexHashedArray"; + } + + size_t getBytesAllocated() const override { return bytes_allocated; } + + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } + + double getFoundRate() const override + { + size_t queries = query_count.load(std::memory_order_relaxed); + if (!queries) + return 0; + return static_cast(found_count.load(std::memory_order_relaxed)) / queries; + } + + double getHitRate() const override { return 1.0; } + + size_t getElementCount() const override { return element_count; } + + double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } + + std::shared_ptr clone() const override + { + return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); + } + + const IDictionarySource * getSource() const override { return source_ptr.get(); } + + const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } + + const DictionaryStructure & getStructure() const override { return dict_struct; } + + bool isInjective(const std::string & attribute_name) const override + { + return dict_struct.getAttribute(attribute_name).injective; + } + + DictionaryKeyType getKeyType() const override { return dictionary_key_type; } + + ColumnPtr getColumn( + const std::string& attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnPtr & default_values_column) const override; + + ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; + + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); } + + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; + + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; + + ColumnPtr getDescendants( + ColumnPtr key_column, + const DataTypePtr & key_type, + size_t level) const override; + + Pipe read(const Names & column_names, size_t max_block_size) const override; + +private: + + using KeyContainerType = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashMap, + HashMapWithSavedHash>>; + + template + using AttributeContainerType = std::conditional_t, std::vector, PaddedPODArray>; + + struct Attribute final + { + AttributeUnderlyingType type; + + std::variant< + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType> + container; + + std::optional> is_index_null; + std::unique_ptr string_arena; + }; + + struct KeyAttribute final + { + + KeyContainerType container; + + }; + + void createAttributes(); + + void blockToAttributes(const Block & block); + + void updateData(); + + void loadData(); + + void calculateBytesAllocated(); + + template + void getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value, + DefaultValueExtractor & default_value_extractor) const; + + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); + + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const; + + void resize(size_t added_rows); + + StringRef copyKeyInArena(StringRef key); + + const DictionaryStructure dict_struct; + const DictionarySourcePtr source_ptr; + const HashedArrayDictionaryStorageConfiguration configuration; + + std::vector attributes; + + KeyAttribute key_attribute; + + size_t bytes_allocated = 0; + size_t element_count = 0; + size_t bucket_count = 0; + mutable std::atomic query_count{0}; + mutable std::atomic found_count{0}; + + BlockPtr update_field_loaded_block; + Arena complex_key_arena; +}; + +extern template class HashedArrayDictionary; +extern template class HashedArrayDictionary; + +} diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 8d24a6ea979..df191edd1c3 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -28,6 +28,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); void registerDictionaryTrie(DictionaryFactory & factory); void registerDictionaryFlat(DictionaryFactory & factory); void registerDictionaryHashed(DictionaryFactory & factory); +void registerDictionaryArrayHashed(DictionaryFactory & factory); void registerDictionaryCache(DictionaryFactory & factory); void registerDictionaryPolygon(DictionaryFactory & factory); void registerDictionaryDirect(DictionaryFactory & factory); @@ -60,6 +61,7 @@ void registerDictionaries() registerDictionaryTrie(factory); registerDictionaryFlat(factory); registerDictionaryHashed(factory); + registerDictionaryArrayHashed(factory); registerDictionaryCache(factory); registerDictionaryPolygon(factory); registerDictionaryDirect(factory); diff --git a/tests/performance/hashed_array_dictionary.xml b/tests/performance/hashed_array_dictionary.xml new file mode 100644 index 00000000000..a26e654248f --- /dev/null +++ b/tests/performance/hashed_array_dictionary.xml @@ -0,0 +1,126 @@ + + + CREATE TABLE simple_key_hashed_array_dictionary_source_table + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE TABLE complex_key_hashed_array_dictionary_source_table + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE DICTIONARY simple_key_hashed_array_dictionary + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_hashed_array_dictionary_source_table')) + LAYOUT(HASHED_ARRAY()) + LIFETIME(MIN 0 MAX 1000); + + + + CREATE DICTIONARY complex_key_hashed_array_dictionary + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id, id_key + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_hashed_array_dictionary_source_table')) + LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) + LIFETIME(MIN 0 MAX 1000); + + + + INSERT INTO simple_key_hashed_array_dictionary_source_table + SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 5000000; + + + + INSERT INTO complex_key_hashed_array_dictionary_source_table + SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 5000000; + + + + + column_name + + 'value_int' + 'value_string' + 'value_decimal' + 'value_string_nullable' + + + + + elements_count + + 5000000 + 7500000 + + + + + + WITH rand64() % toUInt64({elements_count}) as key + SELECT dictGet('default.simple_key_hashed_array_dictionary', {column_name}, key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + WITH rand64() % toUInt64({elements_count}) as key + SELECT dictHas('default.simple_key_hashed_array_dictionary', key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key + SELECT dictGet('default.complex_key_hashed_array_dictionary', {column_name}, key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key + SELECT dictHas('default.complex_key_hashed_array_dictionary', key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + DROP TABLE IF EXISTS simple_key_hashed_array_dictionary_source_table; + DROP TABLE IF EXISTS complex_key_hashed_array_dictionary_source_table; + + DROP DICTIONARY IF EXISTS simple_key_hashed_array_dictionary; + DROP DICTIONARY IF EXISTS complex_key_hashed_array_dictionary; + + diff --git a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference new file mode 100644 index 00000000000..6e88bbad146 --- /dev/null +++ b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference @@ -0,0 +1,66 @@ +Dictionary hashed_array_dictionary_simple_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 value_second_1 +2 value_2 value_second_2 +Dictionary hashed_array_dictionary_simple_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 \N +2 value_2 value_second_2 +Dictionary hashed_array_dictionary_simple_key_hierarchy +dictGet +0 +0 +1 +1 +2 +dictGetHierarchy +[1] +[4,2,1] diff --git a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql new file mode 100644 index 00000000000..8d792836562 --- /dev/null +++ b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql @@ -0,0 +1,125 @@ +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; +CREATE TABLE simple_key_simple_attributes_source_table +( + id UInt64, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_simple_attributes; +CREATE DICTIONARY hashed_array_dictionary_simple_key_simple_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_simple_key_simple_attributes ORDER BY id; + +DROP DICTIONARY hashed_array_dictionary_simple_key_simple_attributes; + +DROP TABLE simple_key_simple_attributes_source_table; + +DROP TABLE IF EXISTS simple_key_complex_attributes_source_table; +CREATE TABLE simple_key_complex_attributes_source_table +( + id UInt64, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL); +INSERT INTO simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_complex_attributes; +CREATE DICTIONARY hashed_array_dictionary_simple_key_complex_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'simple_key_complex_attributes_source_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_simple_key_complex_attributes ORDER BY id; + +DROP DICTIONARY hashed_array_dictionary_simple_key_complex_attributes; +DROP TABLE simple_key_complex_attributes_source_table; + +DROP TABLE IF EXISTS simple_key_hierarchy_table; +CREATE TABLE simple_key_hierarchy_table +( + id UInt64, + parent_id UInt64 +) ENGINE = TinyLog(); + +INSERT INTO simple_key_hierarchy_table VALUES (1, 0); +INSERT INTO simple_key_hierarchy_table VALUES (2, 1); +INSERT INTO simple_key_hierarchy_table VALUES (3, 1); +INSERT INTO simple_key_hierarchy_table VALUES (4, 2); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_hierarchy; +CREATE DICTIONARY hashed_array_dictionary_simple_key_hierarchy +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_hierarchy'; +SELECT 'dictGet'; +SELECT dictGet('hashed_array_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5; +SELECT 'dictGetHierarchy'; +SELECT dictGetHierarchy('hashed_array_dictionary_simple_key_hierarchy', toUInt64(1)); +SELECT dictGetHierarchy('hashed_array_dictionary_simple_key_hierarchy', toUInt64(4)); + +DROP DICTIONARY hashed_array_dictionary_simple_key_hierarchy; +DROP TABLE simple_key_hierarchy_table; diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference new file mode 100644 index 00000000000..ec32fa72b4e --- /dev/null +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference @@ -0,0 +1,56 @@ +Dictionary hashed_array_dictionary_complex_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 value_second_1 +2 id_key_2 value_2 value_second_2 +Dictionary hashed_array_dictionary_complex_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 \N +2 id_key_2 value_2 value_second_2 From 33ef3edcfabd41f2c1968fea08f445a22a730471 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:54:41 +0300 Subject: [PATCH 47/89] Updated tests --- ...99_hashed_array_dictionary_complex_key.sql | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql new file mode 100644 index 00000000000..219b4ab9b71 --- /dev/null +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql @@ -0,0 +1,97 @@ +DROP TABLE IF EXISTS complex_key_simple_attributes_source_table; +CREATE TABLE complex_key_simple_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1'); +INSERT INTO complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_complex_key_simple_attributes; +CREATE DICTIONARY hashed_array_dictionary_complex_key_simple_attributes +( + id UInt64, + id_key String, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_simple_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()); + +SELECT 'Dictionary hashed_array_dictionary_complex_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_complex_key_simple_attributes ORDER BY (id, id_key); + +DROP DICTIONARY hashed_array_dictionary_complex_key_simple_attributes; + +DROP TABLE complex_key_simple_attributes_source_table; + +DROP TABLE IF EXISTS complex_key_complex_attributes_source_table; +CREATE TABLE complex_key_complex_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL); +INSERT INTO complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_complex_key_complex_attributes; +CREATE DICTIONARY hashed_array_dictionary_complex_key_complex_attributes +( + id UInt64, + id_key String, + + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_complex_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(()); + +SELECT 'Dictionary hashed_array_dictionary_complex_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_complex_key_complex_attributes ORDER BY (id, id_key); + +DROP DICTIONARY hashed_array_dictionary_complex_key_complex_attributes; +DROP TABLE complex_key_complex_attributes_source_table; From e66735e21e14406801caa06c7fd0922d3e8a4ca2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:55:28 +0300 Subject: [PATCH 48/89] Fixed style check --- src/Dictionaries/HashedArrayDictionary.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 7160cb67e10..7706f7d6108 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -454,14 +454,15 @@ void HashedArrayDictionary::blockToAttributes(const Block & auto & attribute_container = std::get>(attribute.container); attribute_container.emplace_back(); - if (attribute_is_nullable) { - attribute.is_index_null->emplace_back(); - } - - if (attribute_is_nullable && column_value_to_insert.isNull()) + if (attribute_is_nullable) { - (*attribute.is_index_null).back() = true; - return; + attribute.is_index_null->emplace_back(); + + if (column_value_to_insert.isNull()) + { + (*attribute.is_index_null).back() = true; + return; + } } if constexpr (std::is_same_v) From 01fbd52758c0b72609999e16b4bee34126a6fe8c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 15 Oct 2021 18:52:23 +0200 Subject: [PATCH 49/89] Fix crash with shortcircuit and locardinality in multiIf --- src/Functions/multiIf.cpp | 22 ++++--- ...owcardinality_shortcircuit_crash.reference | 60 +++++++++++++++++++ ...2049_lowcardinality_shortcircuit_crash.sql | 45 ++++++++++++++ 3 files changed, 115 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference create mode 100644 tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 1122d4892c6..3e5242d5f9b 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -124,8 +124,8 @@ public: */ struct Instruction { - const IColumn * condition = nullptr; - const IColumn * source = nullptr; + IColumn::Ptr condition = nullptr; + IColumn::Ptr source = nullptr; bool condition_always_true = false; bool condition_is_nullable = false; @@ -160,15 +160,15 @@ public: } else { - const ColumnWithTypeAndName & cond_col = arguments[i]; + IColumn::Ptr cond_col = arguments[i].column->convertToFullColumnIfLowCardinality(); /// We skip branches that are always false. /// If we encounter a branch that is always true, we can finish. - if (cond_col.column->onlyNull()) + if (cond_col->onlyNull()) continue; - if (const auto * column_const = checkAndGetColumn(*cond_col.column)) + if (const auto * column_const = checkAndGetColumn(*cond_col)) { Field value = column_const->getField(); @@ -181,26 +181,24 @@ public: } else { - if (isColumnNullable(*cond_col.column)) - instruction.condition_is_nullable = true; - - instruction.condition = cond_col.column.get(); + instruction.condition = cond_col; + instruction.condition_is_nullable = instruction.condition->isNullable(); } - instruction.condition_is_short = cond_col.column->size() < arguments[0].column->size(); + instruction.condition_is_short = cond_col->size() < arguments[0].column->size(); } const ColumnWithTypeAndName & source_col = arguments[source_idx]; instruction.source_is_short = source_col.column->size() < arguments[0].column->size(); if (source_col.type->equals(*return_type)) { - instruction.source = source_col.column.get(); + instruction.source = source_col.column; } else { /// Cast all columns to result type. converted_columns_holder.emplace_back(castColumn(source_col, return_type)); - instruction.source = converted_columns_holder.back().get(); + instruction.source = converted_columns_holder.back(); } if (instruction.source && isColumnConst(*instruction.source)) diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference new file mode 100644 index 00000000000..c84236dce7d --- /dev/null +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference @@ -0,0 +1,60 @@ +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 800 +9 900 +10 1000 +11 1100 +12 12000 +13 13000 +14 14000 +15 15000 +16 160000 +17 170000 +18 180000 +19 190000 +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 80000 +9 90000 +10 100000 +11 110000 +12 120000 +13 130000 +14 140000 +15 150000 +16 160000 +17 170000 +18 180000 +19 190000 +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 800 +9 900 +10 1000 +11 1100 +12 12000 +13 13000 +14 14000 +15 15000 +16 160000 +17 170000 +18 180000 +19 190000 diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql new file mode 100644 index 00000000000..2a837380250 --- /dev/null +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql @@ -0,0 +1,45 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/30231 +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10), + CAST(number < 12, 'Nullable(UInt8)'), toString(number * 100), + CAST(number < 16, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; + +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10), + CAST(NULL, 'Nullable(UInt8)'), toString(number * 100), + CAST(NULL, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; + +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10)::LowCardinality(String), + CAST(number < 12, 'Nullable(UInt8)'), toString(number * 100)::Nullable(String), + CAST(number < 16, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000)::LowCardinality(Nullable(String)), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; From ec188000b63aa050a9539b331719469e53749dce Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 17:06:20 +0000 Subject: [PATCH 50/89] Style + more uncomment --- src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- .../fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index f87340b5024..6f6b0d07661 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -30,7 +30,7 @@ add_custom_command( "${CURRENT_DIR_IN_BINARY}/out.cpp" "${CURRENT_DIR_IN_BINARY}/out.proto" COMMAND python3 gen.py clickhouse.g out.cpp out.proto - # DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" ) PROTOBUF_GENERATE_CPP(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 418a5014657..2bc7ad02e87 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -14,13 +14,14 @@ void GenerateSentence(const Sentence&, std::string &, int); -DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { +DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) +{ static std::string input; input.reserve(4096); GenerateSentence(main, input, 0); - if (input.size()) { - + if (input.size()) + { std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); @@ -30,9 +31,8 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); std::cerr << std::endl; - } catch (...) { - } + catch (...) {} input.clear(); } From 882c876090ae88adeaf67db0e6ec1b47f4a2234b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 21:21:08 +0300 Subject: [PATCH 51/89] Update codegen_select_fuzzer.cpp --- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 2bc7ad02e87..6a7a88a8545 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -25,7 +25,8 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); - try { + try + { DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); DB::WriteBufferFromOStream out(std::cerr, 4096); From fceb763228e1268a5589db1a831ebcf929d28524 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:00:30 +0300 Subject: [PATCH 52/89] Fixed test --- .../0_stateless/02099_hashed_array_dictionary_complex_key.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql index 219b4ab9b71..4d2a825c8af 100644 --- a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql @@ -73,7 +73,7 @@ CREATE DICTIONARY hashed_array_dictionary_complex_key_complex_attributes PRIMARY KEY id, id_key SOURCE(CLICKHOUSE(TABLE 'complex_key_complex_attributes_source_table')) LIFETIME(MIN 1 MAX 1000) -LAYOUT(()); +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()); SELECT 'Dictionary hashed_array_dictionary_complex_key_complex_attributes'; SELECT 'dictGet existing value'; From e4ae49e2f4f15d766cb11cbd86f1bd4ac61b6d91 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:16:56 +0300 Subject: [PATCH 53/89] ComplexKeyHashedDictionary fix config parsing --- src/Dictionaries/HashedDictionary.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1f3821096da..234f14a661c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -731,8 +731,18 @@ void registerDictionaryHashed(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - const std::string & layout_prefix = sparse ? ".layout.sparse_hashed" : ".layout.hashed"; - const bool preallocate = config.getBool(config_prefix + layout_prefix + ".preallocate", false); + std::string dictionary_layout_name; + + if (dictionary_key_type == DictionaryKeyType::Simple) + dictionary_layout_name = "hashed"; + else + dictionary_layout_name = "complex_key_hashed"; + + if (sparse) + dictionary_layout_name = "sparse_" + dictionary_layout_name; + + const std::string dictionary_layout_prefix = ".layout." + dictionary_layout_name; + const bool preallocate = config.getBool(config_prefix + dictionary_layout_prefix + ".preallocate", false); HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime}; From fd14faeae22772322346cedda585a2665d9f866d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:18:20 +0300 Subject: [PATCH 54/89] Remove DataStreams folder. --- programs/benchmark/Benchmark.cpp | 4 +-- programs/copier/Internals.h | 2 +- programs/server/Server.cpp | 2 +- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/CMakeLists.txt | 4 +-- src/Client/ClientBase.cpp | 5 ++- src/Client/ClientBase.h | 2 +- src/Client/Connection.cpp | 8 ++--- src/Client/Connection.h | 2 +- src/Client/IServerConnection.h | 4 +-- .../InternalTextLogs.cpp | 2 +- .../InternalTextLogs.h | 0 src/Client/LocalConnection.h | 2 +- .../SquashingTransform.cpp | 2 +- .../SquashingTransform.h | 0 src/Core/Block.cpp | 23 +++++++++++++ src/Core/Block.h | 4 +++ src/Core/SettingsEnums.h | 2 +- src/DataStreams/finalizeBlock.cpp | 27 ---------------- src/DataStreams/finalizeBlock.h | 9 ------ src/DataStreams/materializeBlock.cpp | 29 ----------------- src/DataStreams/materializeBlock.h | 14 -------- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.h | 2 +- .../ClickHouseDictionarySource.cpp | 2 +- .../ExecutableDictionarySource.cpp | 4 +-- .../ExecutablePoolDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- .../MarkInCompressedFile.h | 0 src/{DataStreams => Formats}/NativeReader.cpp | 2 +- src/{DataStreams => Formats}/NativeReader.h | 2 +- src/{DataStreams => Formats}/NativeWriter.cpp | 4 +-- src/{DataStreams => Formats}/NativeWriter.h | 0 .../TemporaryFileStream.cpp | 29 ++--------------- .../TemporaryFileStream.h | 20 +----------- src/{DataStreams => Formats}/formatBlock.cpp | 2 +- src/{DataStreams => Formats}/formatBlock.h | 0 src/Functions/formatRow.cpp | 1 - src/Interpreters/Aggregator.cpp | 3 +- src/Interpreters/Aggregator.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- .../ExecuteScalarSubqueriesVisitor.cpp | 1 - src/Interpreters/HashJoin.cpp | 1 - src/Interpreters/HashJoin.h | 2 +- src/Interpreters/IInterpreter.h | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterExistsQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 -- .../InterpreterShowCreateQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.h | 2 +- src/Interpreters/MergeJoin.cpp | 3 +- src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/ProcessList.h | 2 +- src/Interpreters/Set.h | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 4 +-- src/Interpreters/SortedBlocksWriter.h | 2 +- src/Interpreters/TableJoin.h | 2 +- .../UserDefinedExecutableFunction.cpp | 4 +-- .../UserDefinedExecutableFunction.h | 2 +- .../UserDefinedExecutableFunctionFactory.cpp | 4 +-- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- src/Interpreters/executeQuery.cpp | 3 +- src/Interpreters/executeQuery.h | 2 +- src/Interpreters/join_common.cpp | 2 -- src/NOTICE | 3 -- .../PullingAsyncPipelineExecutor.cpp | 4 +-- .../Executors/PullingAsyncPipelineExecutor.h | 4 +-- .../Executors/PullingPipelineExecutor.cpp | 2 +- .../Executors/PullingPipelineExecutor.h | 4 +-- src/Processors/Formats/IRowInputFormat.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 4 +-- src/Processors/Formats/LazyOutputFormat.h | 6 ++-- src/Processors/Formats/PullingOutputFormat.h | 6 ++-- src/Processors/Pipe.h | 2 +- src/Processors/QueryPlan/AggregatingStep.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.h | 2 +- src/Processors/QueryPlan/CubeStep.h | 2 +- src/Processors/QueryPlan/DistinctStep.h | 2 +- src/Processors/QueryPlan/MergeSortingStep.h | 2 +- .../QueryPlan/MergingAggregatedStep.h | 2 +- src/Processors/QueryPlan/MergingSortedStep.h | 2 +- src/Processors/QueryPlan/OffsetStep.h | 2 +- src/Processors/QueryPlan/PartialSortingStep.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Processors/QueryPlan/RollupStep.h | 2 +- .../QueryPlan/SettingQuotaAndLimitsStep.h | 2 +- src/Processors/Sinks/RemoteSink.h | 27 ++++++++++++++++ .../Sources}/MySQLSource.cpp | 2 +- .../Sources}/MySQLSource.h | 0 src/Processors/Sources/RemoteSource.cpp | 6 ++-- .../Sources}/SQLiteSource.cpp | 0 .../Sources}/SQLiteSource.h | 0 .../Sources}/ShellCommandSource.h | 0 src/Processors/Sources/SourceWithProgress.h | 2 +- .../Sources/TemporaryFileLazySource.cpp | 32 +++++++++++++++++++ .../Sources/TemporaryFileLazySource.h | 28 ++++++++++++++++ .../Transforms/AggregatingTransform.cpp | 3 +- .../Transforms/CreatingSetsTransform.cpp | 1 - .../Transforms/CreatingSetsTransform.h | 2 +- src/Processors/Transforms/DistinctTransform.h | 2 +- .../Transforms/LimitsCheckingTransform.h | 4 +-- .../Transforms/MaterializingTransform.cpp | 1 - .../Transforms/MergeSortingTransform.cpp | 4 +-- .../Transforms/SortingTransform.cpp | 4 +-- .../Transforms/SquashingChunksTransform.h | 2 +- .../Transforms/TotalsHavingTransform.cpp | 21 +++++++++++- .../getSourceFromASTInsertQuery.cpp | 2 +- .../BlockIO.cpp | 2 +- src/{DataStreams => QueryPipeline}/BlockIO.h | 0 .../CMakeLists.txt | 0 .../ConnectionCollector.cpp | 2 +- .../ConnectionCollector.h | 0 .../ExecutionSpeedLimits.cpp | 2 +- .../ExecutionSpeedLimits.h | 2 +- .../ProfileInfo.cpp} | 16 +++++----- .../ProfileInfo.h} | 6 ++-- .../RemoteInserter.cpp} | 2 +- .../RemoteInserter.h} | 20 ------------ .../RemoteQueryExecutor.cpp | 6 ++-- .../RemoteQueryExecutor.h | 4 +-- .../RemoteQueryExecutorReadContext.cpp | 2 +- .../RemoteQueryExecutorReadContext.h | 0 .../SizeLimits.cpp | 2 +- .../SizeLimits.h | 0 .../StreamLocalLimits.h | 4 +-- .../examples/CMakeLists.txt | 0 .../narrowBlockInputStreams.cpp | 0 .../narrowBlockInputStreams.h | 0 .../gtest_blocks_size_merging_streams.cpp | 0 .../tests/gtest_check_sorted_stream.cpp | 0 src/Server/GRPCServer.cpp | 7 ++-- src/Server/TCPHandler.cpp | 6 ++-- src/Server/TCPHandler.h | 8 ++--- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +-- src/Storages/Distributed/DistributedSink.cpp | 4 +-- src/Storages/MarkCache.h | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.h | 4 +-- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageExecutable.cpp | 1 - src/Storages/StorageExecutable.h | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 4 +-- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSet.cpp | 8 ++--- src/Storages/StorageStripeLog.cpp | 4 +-- src/Storages/getStructureOfRemoteTable.cpp | 2 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +- 161 files changed, 311 insertions(+), 337 deletions(-) rename src/{DataStreams => Client}/InternalTextLogs.cpp (99%) rename src/{DataStreams => Client}/InternalTextLogs.h (100%) rename src/{DataStreams => Common}/SquashingTransform.cpp (98%) rename src/{DataStreams => Common}/SquashingTransform.h (100%) delete mode 100644 src/DataStreams/finalizeBlock.cpp delete mode 100644 src/DataStreams/finalizeBlock.h delete mode 100644 src/DataStreams/materializeBlock.cpp delete mode 100644 src/DataStreams/materializeBlock.h rename src/{DataStreams => Formats}/MarkInCompressedFile.h (100%) rename src/{DataStreams => Formats}/NativeReader.cpp (99%) rename src/{DataStreams => Formats}/NativeReader.h (98%) rename src/{DataStreams => Formats}/NativeWriter.cpp (98%) rename src/{DataStreams => Formats}/NativeWriter.h (100%) rename src/{DataStreams => Formats}/TemporaryFileStream.cpp (68%) rename src/{DataStreams => Formats}/TemporaryFileStream.h (63%) rename src/{DataStreams => Formats}/formatBlock.cpp (93%) rename src/{DataStreams => Formats}/formatBlock.h (100%) create mode 100644 src/Processors/Sinks/RemoteSink.h rename src/{Formats => Processors/Sources}/MySQLSource.cpp (99%) rename src/{Formats => Processors/Sources}/MySQLSource.h (100%) rename src/{DataStreams => Processors/Sources}/SQLiteSource.cpp (100%) rename src/{DataStreams => Processors/Sources}/SQLiteSource.h (100%) rename src/{DataStreams => Processors/Sources}/ShellCommandSource.h (100%) create mode 100644 src/Processors/Sources/TemporaryFileLazySource.cpp create mode 100644 src/Processors/Sources/TemporaryFileLazySource.h rename src/{DataStreams => QueryPipeline}/BlockIO.cpp (97%) rename src/{DataStreams => QueryPipeline}/BlockIO.h (100%) rename src/{DataStreams => QueryPipeline}/CMakeLists.txt (100%) rename src/{DataStreams => QueryPipeline}/ConnectionCollector.cpp (98%) rename src/{DataStreams => QueryPipeline}/ConnectionCollector.h (100%) rename src/{DataStreams => QueryPipeline}/ExecutionSpeedLimits.cpp (99%) rename src/{DataStreams => QueryPipeline}/ExecutionSpeedLimits.h (95%) rename src/{DataStreams/BlockStreamProfileInfo.cpp => QueryPipeline/ProfileInfo.cpp} (67%) rename src/{DataStreams/BlockStreamProfileInfo.h => QueryPipeline/ProfileInfo.h} (90%) rename src/{DataStreams/RemoteBlockOutputStream.cpp => QueryPipeline/RemoteInserter.cpp} (98%) rename src/{DataStreams/RemoteBlockOutputStream.h => QueryPipeline/RemoteInserter.h} (56%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutor.cpp (99%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutor.h (98%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutorReadContext.cpp (99%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutorReadContext.h (100%) rename src/{DataStreams => QueryPipeline}/SizeLimits.cpp (97%) rename src/{DataStreams => QueryPipeline}/SizeLimits.h (100%) rename src/{DataStreams => QueryPipeline}/StreamLocalLimits.h (91%) rename src/{DataStreams => QueryPipeline}/examples/CMakeLists.txt (100%) rename src/{DataStreams => QueryPipeline}/narrowBlockInputStreams.cpp (100%) rename src/{DataStreams => QueryPipeline}/narrowBlockInputStreams.h (100%) rename src/{DataStreams => QueryPipeline}/tests/gtest_blocks_size_merging_streams.cpp (100%) rename src/{DataStreams => QueryPipeline}/tests/gtest_check_sorted_stream.cpp (100%) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index caa0a87bde2..1c276a83768 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include #include @@ -432,7 +432,7 @@ private: Progress progress; executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); - BlockStreamProfileInfo info; + ProfileInfo info; while (Block block = executor.read()) info.update(block); diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index a9d8ca726fe..eb2622c6b26 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -49,7 +49,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4ed5b114082..eb4b79e995b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -62,7 +62,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 319c6c344d7..9bc14627ac3 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -1,6 +1,6 @@ #include "LibraryBridgeHelper.h" -#include +#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 08755542ed1..09aaa85c394 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -49,7 +49,7 @@ add_subdirectory (Backups) add_subdirectory (Columns) add_subdirectory (Common) add_subdirectory (Core) -add_subdirectory (DataStreams) +add_subdirectory (QueryPipeline) add_subdirectory (DataTypes) add_subdirectory (Dictionaries) add_subdirectory (Disks) @@ -185,7 +185,7 @@ add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) -add_object_library(clickhouse_datastreams DataStreams) +add_object_library(clickhouse_querypipeline QueryPipeline) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) add_object_library(clickhouse_databases Databases) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e260ed11bd4..f8bed86d7ce 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -47,8 +47,7 @@ #include #include #include -#include -#include +#include namespace fs = std::filesystem; @@ -284,7 +283,7 @@ void ClientBase::onReceiveExceptionFromServer(std::unique_ptr && e) } -void ClientBase::onProfileInfo(const BlockStreamProfileInfo & profile_info) +void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 92f9d48e070..a66e4a0d1ba 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -112,7 +112,7 @@ private: void onTotals(Block & block, ASTPtr parsed_query); void onExtremes(Block & block, ASTPtr parsed_query); void onReceiveExceptionFromServer(std::unique_ptr && e); - void onProfileInfo(const BlockStreamProfileInfo & profile_info); + void onProfileInfo(const ProfileInfo & profile_info); void onEndOfStream(); void sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2aa157bb318..bdaf4ffc76f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -994,9 +994,9 @@ Progress Connection::receiveProgress() const } -BlockStreamProfileInfo Connection::receiveProfileInfo() const +ProfileInfo Connection::receiveProfileInfo() const { - BlockStreamProfileInfo profile_info; + ProfileInfo profile_info; profile_info.read(*in); return profile_info; } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index a5130d876ea..b18b1f1e621 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -253,7 +253,7 @@ private: std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; Progress receiveProgress() const; - BlockStreamProfileInfo receiveProfileInfo() const; + ProfileInfo receiveProfileInfo() const; void initInputBuffers(); void initBlockInput(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 42886c72182..7424afc969d 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include @@ -30,7 +30,7 @@ struct Packet std::unique_ptr exception; std::vector multistring_message; Progress progress; - BlockStreamProfileInfo profile_info; + ProfileInfo profile_info; std::vector part_uuids; Packet() : type(Protocol::Server::Hello) {} diff --git a/src/DataStreams/InternalTextLogs.cpp b/src/Client/InternalTextLogs.cpp similarity index 99% rename from src/DataStreams/InternalTextLogs.cpp rename to src/Client/InternalTextLogs.cpp index a5883d17f28..65592fee670 100644 --- a/src/DataStreams/InternalTextLogs.cpp +++ b/src/Client/InternalTextLogs.cpp @@ -1,4 +1,4 @@ -#include "InternalTextLogs.h" +#include #include #include #include diff --git a/src/DataStreams/InternalTextLogs.h b/src/Client/InternalTextLogs.h similarity index 100% rename from src/DataStreams/InternalTextLogs.h rename to src/Client/InternalTextLogs.h diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index dcea3ed0fc3..5536aeec964 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -2,7 +2,7 @@ #include "Connection.h" #include -#include +#include #include #include diff --git a/src/DataStreams/SquashingTransform.cpp b/src/Common/SquashingTransform.cpp similarity index 98% rename from src/DataStreams/SquashingTransform.cpp rename to src/Common/SquashingTransform.cpp index ea99dc49780..21fa25ed3af 100644 --- a/src/DataStreams/SquashingTransform.cpp +++ b/src/Common/SquashingTransform.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/DataStreams/SquashingTransform.h b/src/Common/SquashingTransform.h similarity index 100% rename from src/DataStreams/SquashingTransform.h rename to src/Common/SquashingTransform.h diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a59ac60155e..40fc5767132 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -707,4 +707,27 @@ ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & column return current_column; } + +Block materializeBlock(const Block & block) +{ + if (!block) + return block; + + Block res = block; + size_t columns = res.columns(); + for (size_t i = 0; i < columns; ++i) + { + auto & element = res.getByPosition(i); + element.column = element.column->convertToFullColumnIfConst(); + } + + return res; +} + +void materializeBlockInplace(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); +} + } diff --git a/src/Core/Block.h b/src/Core/Block.h index a7e3cee194b..e0a032094f6 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -196,4 +196,8 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out /// Properly handles cases, when column is a subcolumn and when it is compressed. ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & column); +/// Converts columns-constants to full columns ("materializes" them). +Block materializeBlock(const Block & block); +void materializeBlockInplace(Block & block); + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index a699da3062c..33c5a6d8645 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/src/DataStreams/finalizeBlock.cpp b/src/DataStreams/finalizeBlock.cpp deleted file mode 100644 index 56068edcc29..00000000000 --- a/src/DataStreams/finalizeBlock.cpp +++ /dev/null @@ -1,27 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - void finalizeBlock(Block & block) - { - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnWithTypeAndName & current = block.getByPosition(i); - const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); - - if (unfinalized_type) - { - current.type = unfinalized_type->getReturnType(); - if (current.column) - { - auto mut_column = IColumn::mutate(std::move(current.column)); - current.column = ColumnAggregateFunction::convertToValues(std::move(mut_column)); - } - } - } - } -} diff --git a/src/DataStreams/finalizeBlock.h b/src/DataStreams/finalizeBlock.h deleted file mode 100644 index 3c81ddae1c7..00000000000 --- a/src/DataStreams/finalizeBlock.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - /// Converts aggregate function columns with non-finalized states to final values - void finalizeBlock(Block & block); -} diff --git a/src/DataStreams/materializeBlock.cpp b/src/DataStreams/materializeBlock.cpp deleted file mode 100644 index 6b47cb87baa..00000000000 --- a/src/DataStreams/materializeBlock.cpp +++ /dev/null @@ -1,29 +0,0 @@ -#include - - -namespace DB -{ - -Block materializeBlock(const Block & block) -{ - if (!block) - return block; - - Block res = block; - size_t columns = res.columns(); - for (size_t i = 0; i < columns; ++i) - { - auto & element = res.getByPosition(i); - element.column = element.column->convertToFullColumnIfConst(); - } - - return res; -} - -void materializeBlockInplace(Block & block) -{ - for (size_t i = 0; i < block.columns(); ++i) - block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); -} - -} diff --git a/src/DataStreams/materializeBlock.h b/src/DataStreams/materializeBlock.h deleted file mode 100644 index 5e1499319c1..00000000000 --- a/src/DataStreams/materializeBlock.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** Converts columns-constants to full columns ("materializes" them). - */ -Block materializeBlock(const Block & block); -void materializeBlockInplace(Block & block); - -} diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index aecbc1474f8..21d927dea77 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index a30341ae927..24a285f11c4 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -11,7 +11,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 618f6bf6d34..ab144761e11 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 514978f2456..8de42760dc2 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 9ec8a9523c6..d2c1195c0c5 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -16,7 +16,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 0cd0701439f..b8c985915dc 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -8,7 +8,7 @@ # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index b563ac797c0..a19eca1fee7 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 91aeda924a1..c09993c2a84 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -4,8 +4,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 79e9b627836..8d1122b1194 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index b7e8468b815..51215b6311b 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 844a6357e29..16e7f483978 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,5 +1,5 @@ #include "HTTPDictionarySource.h" -#include +#include #include #include #include diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index fa26c2b162a..1ecc41036be 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -12,7 +12,7 @@ # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" # include "IDictionarySource.h" -# include +# include namespace Poco { diff --git a/src/DataStreams/MarkInCompressedFile.h b/src/Formats/MarkInCompressedFile.h similarity index 100% rename from src/DataStreams/MarkInCompressedFile.h rename to src/Formats/MarkInCompressedFile.h diff --git a/src/DataStreams/NativeReader.cpp b/src/Formats/NativeReader.cpp similarity index 99% rename from src/DataStreams/NativeReader.cpp rename to src/Formats/NativeReader.cpp index 079dff80eae..2d8fdc160f5 100644 --- a/src/DataStreams/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include diff --git a/src/DataStreams/NativeReader.h b/src/Formats/NativeReader.h similarity index 98% rename from src/DataStreams/NativeReader.h rename to src/Formats/NativeReader.h index 95b03c71764..49c2db7703f 100644 --- a/src/DataStreams/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/DataStreams/NativeWriter.cpp b/src/Formats/NativeWriter.cpp similarity index 98% rename from src/DataStreams/NativeWriter.cpp rename to src/Formats/NativeWriter.cpp index 6e26c443e29..9da0c312362 100644 --- a/src/DataStreams/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -5,8 +5,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/DataStreams/NativeWriter.h b/src/Formats/NativeWriter.h similarity index 100% rename from src/DataStreams/NativeWriter.h rename to src/Formats/NativeWriter.h diff --git a/src/DataStreams/TemporaryFileStream.cpp b/src/Formats/TemporaryFileStream.cpp similarity index 68% rename from src/DataStreams/TemporaryFileStream.cpp rename to src/Formats/TemporaryFileStream.cpp index 4b7c9d50fe7..b19c4aeff35 100644 --- a/src/DataStreams/TemporaryFileStream.cpp +++ b/src/Formats/TemporaryFileStream.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include #include @@ -41,27 +41,4 @@ void TemporaryFileStream::write(const std::string & path, const Block & header, compressed_buf.finalize(); } -TemporaryFileLazySource::TemporaryFileLazySource(const std::string & path_, const Block & header_) - : ISource(header_) - , path(path_) - , done(false) -{} - -Chunk TemporaryFileLazySource::generate() -{ - if (done) - return {}; - - if (!stream) - stream = std::make_unique(path, header); - - auto block = stream->block_in->read(); - if (!block) - { - done = true; - stream.reset(); - } - return Chunk(block.getColumns(), block.rows()); -} - } diff --git a/src/DataStreams/TemporaryFileStream.h b/src/Formats/TemporaryFileStream.h similarity index 63% rename from src/DataStreams/TemporaryFileStream.h rename to src/Formats/TemporaryFileStream.h index e288b5b30fa..5a1e0bc870a 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/Formats/TemporaryFileStream.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { @@ -23,22 +23,4 @@ struct TemporaryFileStream static void write(const std::string & path, const Block & header, QueryPipelineBuilder builder, const std::string & codec); }; - -class TemporaryFileLazySource : public ISource -{ -public: - TemporaryFileLazySource(const std::string & path_, const Block & header_); - String getName() const override { return "TemporaryFileLazySource"; } - -protected: - Chunk generate() override; - -private: - const std::string path; - Block header; - bool done; - - std::unique_ptr stream; -}; - } diff --git a/src/DataStreams/formatBlock.cpp b/src/Formats/formatBlock.cpp similarity index 93% rename from src/DataStreams/formatBlock.cpp rename to src/Formats/formatBlock.cpp index dab321be2e1..3284663fc42 100644 --- a/src/DataStreams/formatBlock.cpp +++ b/src/Formats/formatBlock.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/DataStreams/formatBlock.h b/src/Formats/formatBlock.h similarity index 100% rename from src/DataStreams/formatBlock.h rename to src/Formats/formatBlock.h diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 20341cbe1dc..ee9696cf34f 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 63e3577af55..4f4b981b44d 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -9,8 +9,7 @@ #include #include #include -#include -#include +#include #include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 975075eba96..6d6bf61834b 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -19,7 +19,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index b4ce9f352a2..7e0fa2ba003 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 2b858512b98..2117eec0063 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 156bbfc2d81..fd1c10e8495 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -23,7 +23,6 @@ #include -#include #include #include diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 07fd6d5b89f..b5790c047f4 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -20,7 +20,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 1b4eada3c9f..665a46190fd 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 2ebae17cd6b..3cd39ce7912 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 6ffeef5cc7d..24c30a8be30 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index c29eace1b55..487fa2538c2 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 5f44603a420..e5733a8c28b 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b4ffa15a869..59fd1009381 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,5 +1,3 @@ -#include - #include #include diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index c191a73bc71..30a417f6fa7 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 78c4eca5ca6..e5e447562c6 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -15,7 +15,7 @@ limitations under the License. */ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/InterpreterWatchQuery.h b/src/Interpreters/InterpreterWatchQuery.h index e43ed88af2f..ac167182a71 100644 --- a/src/Interpreters/InterpreterWatchQuery.h +++ b/src/Interpreters/InterpreterWatchQuery.h @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 1fc551334e2..7789c74d596 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -4,8 +4,7 @@ #include #include -#include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 8782a2f7535..0e2e771255d 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 2e300472647..9597c1ee558 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 118779f1935..3146b6af03f 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index ebe4aba71ab..ebec58dcca7 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index 94bebce88f7..ac58ef2ab7b 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 02dcd95ab41..7cd53442ffd 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/UserDefinedExecutableFunction.cpp b/src/Interpreters/UserDefinedExecutableFunction.cpp index 06830df68e6..d57978d0fd6 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.cpp +++ b/src/Interpreters/UserDefinedExecutableFunction.cpp @@ -4,8 +4,8 @@ #include #include -#include -#include +#include +#include namespace DB diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index 240422a02ca..1cb1de47578 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index d6ad2666ff1..cfa1171a84b 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -2,8 +2,8 @@ #include -#include -#include +#include +#include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 0ad40dd3332..e7ec52d03cb 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0a1130c721b..95fb8d38454 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include @@ -49,7 +49,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index a2df9baec73..9c561d8b88c 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index dec925d68c1..bf5d30437ec 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -4,8 +4,6 @@ #include #include -#include - #include #include #include diff --git a/src/NOTICE b/src/NOTICE index d0d3efe3f8e..c68280b1529 100644 --- a/src/NOTICE +++ b/src/NOTICE @@ -18,9 +18,6 @@ Common/UInt128.h Core/Block.h Core/Defines.h Core/Settings.h -DataStreams/PushingToViewsBlockOutputStream.cpp -DataStreams/PushingToViewsBlockOutputStream.h -DataStreams/copyData.cpp Databases/DatabasesCommon.cpp IO/WriteBufferValidUTF8.cpp Interpreters/InterpreterAlterQuery.cpp diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 819344e4225..80ed8225c79 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -225,12 +225,12 @@ Block PullingAsyncPipelineExecutor::getExtremesBlock() return header.cloneWithColumns(extremes.detachColumns()); } -BlockStreamProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo() +ProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo() { if (lazy_format) return lazy_format->getProfileInfo(); - static BlockStreamProfileInfo profile_info; + static ProfileInfo profile_info; static std::once_flag flag; /// Calculate rows before limit here to avoid race. std::call_once(flag, []() { profile_info.getRowsBeforeLimit(); }); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 2ce75aecab7..7e45246ffd6 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -8,7 +8,7 @@ class QueryPipeline; class Block; class Chunk; class LazyOutputFormat; -struct BlockStreamProfileInfo; +struct ProfileInfo; /// Asynchronous pulling executor for QueryPipeline. /// Always creates extra thread. If query is executed in single thread, use PullingPipelineExecutor. @@ -44,7 +44,7 @@ public: Block getExtremesBlock(); /// Get query profile info. - BlockStreamProfileInfo & getProfileInfo(); + ProfileInfo & getProfileInfo(); /// Internal executor data. struct Data; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 7da2a6d3059..ad7da63b0b1 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -118,7 +118,7 @@ Block PullingPipelineExecutor::getExtremesBlock() return header.cloneWithColumns(extremes.detachColumns()); } -BlockStreamProfileInfo & PullingPipelineExecutor::getProfileInfo() +ProfileInfo & PullingPipelineExecutor::getProfileInfo() { return pulling_format->getProfileInfo(); } diff --git a/src/Processors/Executors/PullingPipelineExecutor.h b/src/Processors/Executors/PullingPipelineExecutor.h index 878d66bd3d4..e05f4f3738d 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.h +++ b/src/Processors/Executors/PullingPipelineExecutor.h @@ -10,7 +10,7 @@ class Chunk; class QueryPipeline; class PipelineExecutor; class PullingOutputFormat; -struct BlockStreamProfileInfo; +struct ProfileInfo; using PipelineExecutorPtr = std::shared_ptr; @@ -46,7 +46,7 @@ public: Block getExtremesBlock(); /// Get query profile info. - BlockStreamProfileInfo & getProfileInfo(); + ProfileInfo & getProfileInfo(); private: std::atomic_bool has_data_flag = false; diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 19a94d41044..87caadd93da 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include class Stopwatch; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 5daa38967b9..07cf4670981 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 2c29f55c4f3..82a0cb2fc07 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB @@ -25,7 +25,7 @@ public: bool isFinished() { return finished_processing && queue.size() == 0; } - BlockStreamProfileInfo & getProfileInfo() { return info; } + ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; @@ -65,7 +65,7 @@ private: /// Is not used. static WriteBuffer out; - BlockStreamProfileInfo info; + ProfileInfo info; std::atomic finished_processing; }; diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index 53b2086712f..a231b7679f3 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { @@ -20,7 +20,7 @@ public: Chunk getTotals(); Chunk getExtremes(); - BlockStreamProfileInfo & getProfileInfo() { return info; } + ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; @@ -38,7 +38,7 @@ private: std::atomic_bool & has_data_flag; - BlockStreamProfileInfo info; + ProfileInfo info; /// Is not used. static WriteBuffer out; diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index a07c68f56b2..3341734430c 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 8583e5be485..c7d67c75894 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 8d20c764e8a..e20c28e10f4 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.h b/src/Processors/QueryPlan/CubeStep.h index 45077d78a90..1079bed5398 100644 --- a/src/Processors/QueryPlan/CubeStep.h +++ b/src/Processors/QueryPlan/CubeStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index b08e93dffa9..a48a779425d 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h index d5daa041256..947ced829c6 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ b/src/Processors/QueryPlan/MergeSortingStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 9171512571a..eeead41b5f9 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h index 5d27e59ab76..e886de42ca8 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ b/src/Processors/QueryPlan/MergingSortedStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/OffsetStep.h b/src/Processors/QueryPlan/OffsetStep.h index 488c55b6460..f16559bcfad 100644 --- a/src/Processors/QueryPlan/OffsetStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h index bd8fd30ce02..9b7b8e8baa5 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ b/src/Processors/QueryPlan/PartialSortingStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cd2f42ece58..399e7d01839 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/RollupStep.h b/src/Processors/QueryPlan/RollupStep.h index 2ff3040d7a7..7cd71fecdc1 100644 --- a/src/Processors/QueryPlan/RollupStep.h +++ b/src/Processors/QueryPlan/RollupStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h index b36ddfb3768..a8d1eef4b08 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Sinks/RemoteSink.h b/src/Processors/Sinks/RemoteSink.h new file mode 100644 index 00000000000..30cf958c072 --- /dev/null +++ b/src/Processors/Sinks/RemoteSink.h @@ -0,0 +1,27 @@ +#pragma once +#include +#include + +namespace DB +{ + +class RemoteSink final : public RemoteInserter, public SinkToStorage +{ +public: + explicit RemoteSink( + Connection & connection_, + const ConnectionTimeouts & timeouts, + const String & query_, + const Settings & settings_, + const ClientInfo & client_info_) + : RemoteInserter(connection_, timeouts, query_, settings_, client_info_) + , SinkToStorage(RemoteInserter::getHeader()) + { + } + + String getName() const override { return "RemoteSink"; } + void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } + void onFinish() override { RemoteInserter::onFinish(); } +}; + +} diff --git a/src/Formats/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp similarity index 99% rename from src/Formats/MySQLSource.cpp rename to src/Processors/Sources/MySQLSource.cpp index 069aa0cb56b..5bda662466c 100644 --- a/src/Formats/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -19,7 +19,7 @@ #include #include #include -#include "MySQLSource.h" +#include namespace DB diff --git a/src/Formats/MySQLSource.h b/src/Processors/Sources/MySQLSource.h similarity index 100% rename from src/Formats/MySQLSource.h rename to src/Processors/Sources/MySQLSource.h diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index bf3ef32214d..99ba459cf2c 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -1,6 +1,6 @@ #include -#include -#include +#include +#include #include #include @@ -56,7 +56,7 @@ std::optional RemoteSource::tryGenerate() query_executor->setProgressCallback([this](const Progress & value) { progress(value); }); /// Get rows_before_limit result for remote query from ProfileInfo packet. - query_executor->setProfileInfoCallback([this](const BlockStreamProfileInfo & info) + query_executor->setProfileInfoCallback([this](const ProfileInfo & info) { if (rows_before_limit && info.hasAppliedLimit()) rows_before_limit->set(info.getRowsBeforeLimit()); diff --git a/src/DataStreams/SQLiteSource.cpp b/src/Processors/Sources/SQLiteSource.cpp similarity index 100% rename from src/DataStreams/SQLiteSource.cpp rename to src/Processors/Sources/SQLiteSource.cpp diff --git a/src/DataStreams/SQLiteSource.h b/src/Processors/Sources/SQLiteSource.h similarity index 100% rename from src/DataStreams/SQLiteSource.h rename to src/Processors/Sources/SQLiteSource.h diff --git a/src/DataStreams/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h similarity index 100% rename from src/DataStreams/ShellCommandSource.h rename to src/Processors/Sources/ShellCommandSource.h diff --git a/src/Processors/Sources/SourceWithProgress.h b/src/Processors/Sources/SourceWithProgress.h index bf57c3b013b..912a548f977 100644 --- a/src/Processors/Sources/SourceWithProgress.h +++ b/src/Processors/Sources/SourceWithProgress.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include diff --git a/src/Processors/Sources/TemporaryFileLazySource.cpp b/src/Processors/Sources/TemporaryFileLazySource.cpp new file mode 100644 index 00000000000..0382229a7c0 --- /dev/null +++ b/src/Processors/Sources/TemporaryFileLazySource.cpp @@ -0,0 +1,32 @@ +#include +#include + +namespace DB +{ + +TemporaryFileLazySource::~TemporaryFileLazySource() = default; + +TemporaryFileLazySource::TemporaryFileLazySource(const std::string & path_, const Block & header_) + : ISource(header_) + , path(path_) + , done(false) +{} + +Chunk TemporaryFileLazySource::generate() +{ + if (done) + return {}; + + if (!stream) + stream = std::make_unique(path, header); + + auto block = stream->block_in->read(); + if (!block) + { + done = true; + stream.reset(); + } + return Chunk(block.getColumns(), block.rows()); +} + +} diff --git a/src/Processors/Sources/TemporaryFileLazySource.h b/src/Processors/Sources/TemporaryFileLazySource.h new file mode 100644 index 00000000000..b2e9d5d5500 --- /dev/null +++ b/src/Processors/Sources/TemporaryFileLazySource.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +struct TemporaryFileStream; + +class TemporaryFileLazySource : public ISource +{ +public: + TemporaryFileLazySource(const std::string & path_, const Block & header_); + ~TemporaryFileLazySource() override; + String getName() const override { return "TemporaryFileLazySource"; } + +protected: + Chunk generate() override; + +private: + const std::string path; + Block header; + bool done; + + std::unique_ptr stream; +}; + +} diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 9011d188b81..bf3cafd6ff5 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -1,10 +1,9 @@ #include -#include +#include #include #include #include -#include #include namespace ProfileEvents diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e6ae620e69b..fb3c8d6a87b 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -7,7 +7,6 @@ #include #include -#include namespace DB diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index a5a67e99afc..8e7a09b320d 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Processors/Transforms/DistinctTransform.h b/src/Processors/Transforms/DistinctTransform.h index 236f9026c63..d80fdb5bc22 100644 --- a/src/Processors/Transforms/DistinctTransform.h +++ b/src/Processors/Transforms/DistinctTransform.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/Transforms/LimitsCheckingTransform.h b/src/Processors/Transforms/LimitsCheckingTransform.h index 9de5cbf5125..50891ece654 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/src/Processors/Transforms/LimitsCheckingTransform.h @@ -1,10 +1,10 @@ #pragma once #include -#include +#include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index f13d5376ebe..abf416e8047 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 6e379a3c4ba..e272fd0f183 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -6,8 +6,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 2c9098adaa6..eeb576731ab 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -9,8 +9,8 @@ #include #include -#include -#include +#include +#include namespace ProfileEvents diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 75a799e5af1..45a0f33d666 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index f30058fadb9..0b7797da24f 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include namespace DB @@ -29,6 +29,25 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } +void finalizeBlock(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + { + ColumnWithTypeAndName & current = block.getByPosition(i); + const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); + + if (unfinalized_type) + { + current.type = unfinalized_type->getReturnType(); + if (current.column) + { + auto mut_column = IColumn::mutate(std::move(current.column)); + current.column = ColumnAggregateFunction::convertToValues(std::move(mut_column)); + } + } + } +} + Block TotalsHavingTransform::transformHeader( Block block, const ActionsDAG * expression, diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index a1cce03a1a5..57cfa6c01b8 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataStreams/BlockIO.cpp b/src/QueryPipeline/BlockIO.cpp similarity index 97% rename from src/DataStreams/BlockIO.cpp rename to src/QueryPipeline/BlockIO.cpp index 692b69388ea..671ba6e4c39 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/QueryPipeline/BlockIO.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/DataStreams/BlockIO.h b/src/QueryPipeline/BlockIO.h similarity index 100% rename from src/DataStreams/BlockIO.h rename to src/QueryPipeline/BlockIO.h diff --git a/src/DataStreams/CMakeLists.txt b/src/QueryPipeline/CMakeLists.txt similarity index 100% rename from src/DataStreams/CMakeLists.txt rename to src/QueryPipeline/CMakeLists.txt diff --git a/src/DataStreams/ConnectionCollector.cpp b/src/QueryPipeline/ConnectionCollector.cpp similarity index 98% rename from src/DataStreams/ConnectionCollector.cpp rename to src/QueryPipeline/ConnectionCollector.cpp index 8e700c0ab7f..65b030fd623 100644 --- a/src/DataStreams/ConnectionCollector.cpp +++ b/src/QueryPipeline/ConnectionCollector.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ConnectionCollector.h b/src/QueryPipeline/ConnectionCollector.h similarity index 100% rename from src/DataStreams/ConnectionCollector.h rename to src/QueryPipeline/ConnectionCollector.h diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp similarity index 99% rename from src/DataStreams/ExecutionSpeedLimits.cpp rename to src/QueryPipeline/ExecutionSpeedLimits.cpp index e340ee71ab0..2738903cedb 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ExecutionSpeedLimits.h b/src/QueryPipeline/ExecutionSpeedLimits.h similarity index 95% rename from src/DataStreams/ExecutionSpeedLimits.h rename to src/QueryPipeline/ExecutionSpeedLimits.h index b8c320bb005..63658462c9f 100644 --- a/src/DataStreams/ExecutionSpeedLimits.h +++ b/src/QueryPipeline/ExecutionSpeedLimits.h @@ -2,7 +2,7 @@ #include #include -#include +#include class Stopwatch; diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp similarity index 67% rename from src/DataStreams/BlockStreamProfileInfo.cpp rename to src/QueryPipeline/ProfileInfo.cpp index 9a06d905223..32986f7259c 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -8,7 +8,7 @@ namespace DB { -void BlockStreamProfileInfo::read(ReadBuffer & in) +void ProfileInfo::read(ReadBuffer & in) { readVarUInt(rows, in); readVarUInt(blocks, in); @@ -19,7 +19,7 @@ void BlockStreamProfileInfo::read(ReadBuffer & in) } -void BlockStreamProfileInfo::write(WriteBuffer & out) const +void ProfileInfo::write(WriteBuffer & out) const { writeVarUInt(rows, out); writeVarUInt(blocks, out); @@ -30,7 +30,7 @@ void BlockStreamProfileInfo::write(WriteBuffer & out) const } -void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool skip_block_size_info) +void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) { if (!skip_block_size_info) { @@ -44,24 +44,24 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk } -size_t BlockStreamProfileInfo::getRowsBeforeLimit() const +size_t ProfileInfo::getRowsBeforeLimit() const { return rows_before_limit; } -bool BlockStreamProfileInfo::hasAppliedLimit() const +bool ProfileInfo::hasAppliedLimit() const { return applied_limit; } -void BlockStreamProfileInfo::update(Block & block) +void ProfileInfo::update(Block & block) { update(block.rows(), block.bytes()); } -void BlockStreamProfileInfo::update(size_t num_rows, size_t num_bytes) +void ProfileInfo::update(size_t num_rows, size_t num_bytes) { ++blocks; rows += num_rows; diff --git a/src/DataStreams/BlockStreamProfileInfo.h b/src/QueryPipeline/ProfileInfo.h similarity index 90% rename from src/DataStreams/BlockStreamProfileInfo.h rename to src/QueryPipeline/ProfileInfo.h index 1707b941445..335092ce244 100644 --- a/src/DataStreams/BlockStreamProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -13,7 +13,7 @@ class ReadBuffer; class WriteBuffer; /// Information for profiling. See IBlockInputStream.h -struct BlockStreamProfileInfo +struct ProfileInfo { bool started = false; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time @@ -22,7 +22,7 @@ struct BlockStreamProfileInfo size_t blocks = 0; size_t bytes = 0; - using BlockStreamProfileInfos = std::vector; + using ProfileInfos = std::vector; /** Get the number of rows if there were no LIMIT. * If there is no LIMIT, 0 is returned. @@ -42,7 +42,7 @@ struct BlockStreamProfileInfo /// Sets main fields from other object (see methods above). /// If skip_block_size_info if true, then rows, bytes and block fields are ignored. - void setFrom(const BlockStreamProfileInfo & rhs, bool skip_block_size_info); + void setFrom(const ProfileInfo & rhs, bool skip_block_size_info); /// Only for Processors. void setRowsBeforeLimit(size_t rows_before_limit_) diff --git a/src/DataStreams/RemoteBlockOutputStream.cpp b/src/QueryPipeline/RemoteInserter.cpp similarity index 98% rename from src/DataStreams/RemoteBlockOutputStream.cpp rename to src/QueryPipeline/RemoteInserter.cpp index 7642098ff0c..c34c625dc6d 100644 --- a/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/RemoteBlockOutputStream.h b/src/QueryPipeline/RemoteInserter.h similarity index 56% rename from src/DataStreams/RemoteBlockOutputStream.h rename to src/QueryPipeline/RemoteInserter.h index f1f49015c9d..0688b555825 100644 --- a/src/DataStreams/RemoteBlockOutputStream.h +++ b/src/QueryPipeline/RemoteInserter.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -44,23 +43,4 @@ private: bool finished = false; }; -class RemoteSink final : public RemoteInserter, public SinkToStorage -{ -public: - explicit RemoteSink( - Connection & connection_, - const ConnectionTimeouts & timeouts, - const String & query_, - const Settings & settings_, - const ClientInfo & client_info_) - : RemoteInserter(connection_, timeouts, query_, settings_, client_info_) - , SinkToStorage(RemoteInserter::getHeader()) - { - } - - String getName() const override { return "RemoteSink"; } - void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } - void onFinish() override { RemoteInserter::onFinish(); } -}; - } diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp similarity index 99% rename from src/DataStreams/RemoteQueryExecutor.cpp rename to src/QueryPipeline/RemoteQueryExecutor.cpp index 3c78fddfd39..23fe3a5f220 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include diff --git a/src/DataStreams/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h similarity index 98% rename from src/DataStreams/RemoteQueryExecutor.h rename to src/QueryPipeline/RemoteQueryExecutor.h index d82f9983894..b7a2509ea97 100644 --- a/src/DataStreams/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -21,8 +21,8 @@ using ThrottlerPtr = std::shared_ptr; struct Progress; using ProgressCallback = std::function; -struct BlockStreamProfileInfo; -using ProfileInfoCallback = std::function; +struct ProfileInfo; +using ProfileInfoCallback = std::function; class RemoteQueryExecutorReadContext; diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp similarity index 99% rename from src/DataStreams/RemoteQueryExecutorReadContext.cpp rename to src/QueryPipeline/RemoteQueryExecutorReadContext.cpp index 6bdf52d2831..5f7b5e24967 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp @@ -1,6 +1,6 @@ #if defined(OS_LINUX) -#include +#include #include #include #include diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h similarity index 100% rename from src/DataStreams/RemoteQueryExecutorReadContext.h rename to src/QueryPipeline/RemoteQueryExecutorReadContext.h diff --git a/src/DataStreams/SizeLimits.cpp b/src/QueryPipeline/SizeLimits.cpp similarity index 97% rename from src/DataStreams/SizeLimits.cpp rename to src/QueryPipeline/SizeLimits.cpp index 06dde923e55..90005902f67 100644 --- a/src/DataStreams/SizeLimits.cpp +++ b/src/QueryPipeline/SizeLimits.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/DataStreams/SizeLimits.h b/src/QueryPipeline/SizeLimits.h similarity index 100% rename from src/DataStreams/SizeLimits.h rename to src/QueryPipeline/SizeLimits.h diff --git a/src/DataStreams/StreamLocalLimits.h b/src/QueryPipeline/StreamLocalLimits.h similarity index 91% rename from src/DataStreams/StreamLocalLimits.h rename to src/QueryPipeline/StreamLocalLimits.h index efda6a941cc..7f49a5d0b07 100644 --- a/src/DataStreams/StreamLocalLimits.h +++ b/src/QueryPipeline/StreamLocalLimits.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include namespace DB { diff --git a/src/DataStreams/examples/CMakeLists.txt b/src/QueryPipeline/examples/CMakeLists.txt similarity index 100% rename from src/DataStreams/examples/CMakeLists.txt rename to src/QueryPipeline/examples/CMakeLists.txt diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/QueryPipeline/narrowBlockInputStreams.cpp similarity index 100% rename from src/DataStreams/narrowBlockInputStreams.cpp rename to src/QueryPipeline/narrowBlockInputStreams.cpp diff --git a/src/DataStreams/narrowBlockInputStreams.h b/src/QueryPipeline/narrowBlockInputStreams.h similarity index 100% rename from src/DataStreams/narrowBlockInputStreams.h rename to src/QueryPipeline/narrowBlockInputStreams.h diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp similarity index 100% rename from src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp rename to src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp diff --git a/src/DataStreams/tests/gtest_check_sorted_stream.cpp b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp similarity index 100% rename from src/DataStreams/tests/gtest_check_sorted_stream.cpp rename to src/QueryPipeline/tests/gtest_check_sorted_stream.cpp diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 5d62a295dad..5b727253dff 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -9,8 +9,7 @@ #include #include #include -#include -#include +#include #include #include #include @@ -595,7 +594,7 @@ namespace void addProgressToResult(); void addTotalsToResult(const Block & totals); void addExtremesToResult(const Block & extremes); - void addProfileInfoToResult(const BlockStreamProfileInfo & info); + void addProfileInfoToResult(const ProfileInfo & info); void addLogsToResult(); void sendResult(); void throwIfFailedToSendResult(); @@ -1381,7 +1380,7 @@ namespace format->doWriteSuffix(); } - void Call::addProfileInfoToResult(const BlockStreamProfileInfo & info) + void Call::addProfileInfoToResult(const ProfileInfo & info) { auto & stats = *result.mutable_stats(); stats.set_rows(info.rows); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f3247e7bc2b..4e8291f5281 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -16,8 +16,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -772,7 +772,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() out->next(); } -void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info) +void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index e89d82cfcc8..cb14323906b 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -9,10 +9,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include "IServer.h" @@ -30,7 +30,7 @@ namespace DB class Session; struct Settings; class ColumnsDescription; -struct BlockStreamProfileInfo; +struct ProfileInfo; /// State of query processing. struct QueryState @@ -224,7 +224,7 @@ private: void sendEndOfStream(); void sendPartUUIDs(); void sendReadTaskRequestAssumeLocked(); - void sendProfileInfo(const BlockStreamProfileInfo & info); + void sendProfileInfo(const ProfileInfo & info); void sendTotals(const Block & totals); void sendExtremes(const Block & extremes); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 167e36ebbe3..254d82520dc 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 1841be22b72..cf349d1f8cf 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -14,8 +14,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index ccf8a2e606d..06143e954f8 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace ProfileEvents diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 9e09cd0036e..a1df4a13e6a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 7624dc303e0..4fec5ce46bc 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index da79b917b5a..d128ecb9e27 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index df92b270542..1dd75f09b92 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -4,7 +4,7 @@ #include -#include +#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 0f47f654428..21784952c23 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 0d17e003ce4..74df17f1463 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6492c9e07c0..38db0b61e8d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3bdf3218b2e..a1724af38cb 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d95a9465bd6..c94c519c3b9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 0e7faad194e..8f7654821cb 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -23,11 +23,11 @@ #include #include #include -#include +#include #include #include "Processors/Sources/SourceWithProgress.h" #include -#include +#include #include #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d7eef35e60b..4e2c6cfbe10 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -3,7 +3,7 @@ #if USE_SQLITE #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 2547af1b0ad..77231ce49fd 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -4,9 +4,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include #include #include @@ -215,7 +215,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) CompressedReadBuffer compressed_backup_buf(*backup_buf); NativeReader backup_stream(compressed_backup_buf, 0); - BlockStreamProfileInfo info; + ProfileInfo info; while (Block block = backup_stream.read()) { info.update(block); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0cd07afc26c..66f49761793 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -14,8 +14,8 @@ #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 639692beda5..532abb8e2f3 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 005a689f895..80f108eb68a 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 7bd8ad2e740..7e28decfdb0 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include #include From da45d55e22b141eb3764ef9eb9812ecaf258e641 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:18:37 +0300 Subject: [PATCH 55/89] Remove DataStreams folder. --- utils/wal-dump/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/wal-dump/main.cpp b/utils/wal-dump/main.cpp index 0e47c39fb5a..3566936324b 100644 --- a/utils/wal-dump/main.cpp +++ b/utils/wal-dump/main.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include From 6f682d54b2ee0a27465dcebf23c43f4ee1934276 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:21:18 +0300 Subject: [PATCH 56/89] Fixed test --- src/Dictionaries/HashedDictionary.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index eec7cec0285..917eb4cbde3 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -664,10 +664,7 @@ Pipe HashedDictionary::read(const Names & column_na }); } - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); - else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template From f3cbac79d343c0fa4e697fbbd40bc7ca0346eeb2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:22:24 +0300 Subject: [PATCH 57/89] Try fix integration test. --- src/DataStreams/BlockStreamProfileInfo.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/DataStreams/BlockStreamProfileInfo.cpp index 9a06d905223..05c1ac96db0 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/DataStreams/BlockStreamProfileInfo.cpp @@ -46,12 +46,14 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk size_t BlockStreamProfileInfo::getRowsBeforeLimit() const { + calculated_rows_before_limit = true; return rows_before_limit; } bool BlockStreamProfileInfo::hasAppliedLimit() const { + calculated_rows_before_limit = true; return applied_limit; } From a35d7096d8117d1182dcf0d4951d3ab781fbc84d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:26:09 +0300 Subject: [PATCH 58/89] Added concept HasIndexOperator --- src/Functions/FunctionsJSON.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 9558b856511..cb55ba6b83b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -58,12 +58,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template -struct HasIndexOperator : std::false_type {}; - -template -struct HasIndexOperator()[0])>> : std::true_type {}; - +template +concept HasIndexOperator = requires (T t) +{ + t[0]; +}; /// Functions to parse JSONs and extract values from it. /// The first argument of all these functions gets a JSON, @@ -285,7 +284,7 @@ private: return true; } - if constexpr (HasIndexOperator::value) + if constexpr (HasIndexOperator) { if (element.isObject()) { From 41acc524581782d54c6985915c5381f1badaef18 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 00:12:51 +0300 Subject: [PATCH 59/89] Preparation to build with Musl --- base/base/LineReader.cpp | 4 +++ .../include/jemalloc/jemalloc_defs.h | 6 +++-- .../include/jemalloc/jemalloc_protos.h | 2 +- .../internal/jemalloc_internal_defs.h.in | 14 +++++----- contrib/libcxx-cmake/CMakeLists.txt | 4 +++ .../linux_x86_64/include/portable.h | 4 ++- src/Common/QueryProfiler.cpp | 8 +++--- src/Common/malloc.cpp | 4 +++ src/IO/BitHelpers.h | 26 +++++-------------- 9 files changed, 40 insertions(+), 32 deletions(-) diff --git a/base/base/LineReader.cpp b/base/base/LineReader.cpp index 8600f4c7b65..d325154ee61 100644 --- a/base/base/LineReader.cpp +++ b/base/base/LineReader.cpp @@ -5,6 +5,10 @@ #include #include +#include +#include +#include + #ifdef OS_LINUX /// We can detect if code is linked with one or another readline variants or open the library dynamically. diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h index 6a03a231a0e..0aa4033f859 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h @@ -18,8 +18,10 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE_MEMALIGN -#define JEMALLOC_OVERRIDE_VALLOC +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE_MEMALIGN + #define JEMALLOC_OVERRIDE_VALLOC +#endif /* * At least Linux omits the "const" in: diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h index 8506237729d..2e35e7b6249 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h @@ -1,6 +1,6 @@ // OSX does not have this for system alloc functions, so you will get // "exception specification in declaration" error. -#if defined(__APPLE__) || defined(__FreeBSD__) +#if defined(__APPLE__) || defined(__FreeBSD__) || defined(USE_MUSL) # undef JEMALLOC_NOTHROW # define JEMALLOC_NOTHROW diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index d5cf0e719ef..44ff2d9fad1 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -13,12 +13,14 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE___LIBC_CALLOC -#define JEMALLOC_OVERRIDE___LIBC_FREE -#define JEMALLOC_OVERRIDE___LIBC_MALLOC -#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN -#define JEMALLOC_OVERRIDE___LIBC_REALLOC -#define JEMALLOC_OVERRIDE___LIBC_VALLOC +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif /* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index 0cfb4191619..ac67f2563a3 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -56,6 +56,10 @@ if (USE_UNWIND) target_compile_definitions(cxx PUBLIC -DSTD_EXCEPTION_HAS_STACK_TRACE=1) endif () +if (USE_MUSL) + target_compile_definitions(cxx PUBLIC -D_LIBCPP_HAS_MUSL_LIBC=1) +endif () + # Override the deduced attribute support that causes error. if (OS_DARWIN AND COMPILER_GCC) add_compile_definitions(_LIBCPP_INIT_PRIORITY_MAX) diff --git a/contrib/openldap-cmake/linux_x86_64/include/portable.h b/contrib/openldap-cmake/linux_x86_64/include/portable.h index 2924b6713a4..ab7052bda91 100644 --- a/contrib/openldap-cmake/linux_x86_64/include/portable.h +++ b/contrib/openldap-cmake/linux_x86_64/include/portable.h @@ -98,7 +98,9 @@ #define HAVE_BCOPY 1 /* Define to 1 if you have the header file. */ -#define HAVE_BITS_TYPES_H 1 +#if !defined(USE_MUSL) + #define HAVE_BITS_TYPES_H 1 +#endif /* Define to 1 if you have the `chroot' function. */ #define HAVE_CHROOT 1 diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 7b905937e11..aa40226093a 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -124,11 +124,13 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const sev.sigev_notify = SIGEV_THREAD_ID; sev.sigev_signo = pause_signal; -# if defined(OS_FREEBSD) +#if defined(OS_FREEBSD) sev._sigev_un._threadid = thread_id; -# else +#elif defined(USE_MUSL) + sev.sigev_notify_thread_id = thread_id; +#else sev._sigev_un._tid = thread_id; -# endif +#endif if (timer_create(clock_type, &sev, &timer_id)) { /// In Google Cloud Run, the function "timer_create" is implemented incorrectly as of 2020-01-25. diff --git a/src/Common/malloc.cpp b/src/Common/malloc.cpp index ec472d5d1d6..88281d9c80e 100644 --- a/src/Common/malloc.cpp +++ b/src/Common/malloc.cpp @@ -17,7 +17,9 @@ extern "C" void *aligned_alloc(size_t alignment, size_t size); void *valloc(size_t size); void *memalign(size_t alignment, size_t size); +#if !defined(USE_MUSL) void *pvalloc(size_t size); +#endif } #pragma GCC diagnostic pop @@ -39,6 +41,8 @@ static void dummyFunctionForInterposing() ignore(aligned_alloc(0, 0)); // -V575 NOLINT ignore(valloc(0)); // -V575 NOLINT ignore(memalign(0, 0)); // -V575 NOLINT +#if !defined(USE_MUSL) ignore(pvalloc(0)); // -V575 NOLINT +#endif } #endif diff --git a/src/IO/BitHelpers.h b/src/IO/BitHelpers.h index bcc36305021..d15297637a3 100644 --- a/src/IO/BitHelpers.h +++ b/src/IO/BitHelpers.h @@ -7,17 +7,6 @@ #include #include -#if defined(__OpenBSD__) || defined(__FreeBSD__) || defined (__ANDROID__) -# include -#elif defined(__sun) -# include -#elif defined(__APPLE__) -# include - -# define htobe64(x) OSSwapHostToBigInt64(x) -# define be64toh(x) OSSwapBigToHostInt64(x) -#endif - namespace DB { @@ -152,7 +141,7 @@ private: memcpy(&tmp_buffer, source_current, bytes_to_read); source_current += bytes_to_read; - tmp_buffer = be64toh(tmp_buffer); + tmp_buffer = __builtin_bswap64(tmp_buffer); bits_buffer |= BufferType(tmp_buffer) << ((sizeof(BufferType) - sizeof(tmp_buffer)) * 8 - bits_count); bits_count += static_cast(bytes_to_read) * 8; @@ -200,7 +189,7 @@ public: capacity = BIT_BUFFER_SIZE - bits_count; } -// write low bits of value as high bits of bits_buffer + // write low bits of value as high bits of bits_buffer const UInt64 mask = maskLowBits(bits_to_write); BufferType v = value & mask; v <<= capacity - bits_to_write; @@ -212,7 +201,7 @@ public: // flush contents of bits_buffer to the dest_current, partial bytes are completed with zeroes. inline void flush() { - bits_count = (bits_count + 8 - 1) & ~(8 - 1); // align UP to 8-bytes, so doFlush will write ALL data from bits_buffer + bits_count = (bits_count + 8 - 1) & ~(8 - 1); // align up to 8-bytes, so doFlush will write all data from bits_buffer while (bits_count != 0) doFlush(); } @@ -231,13 +220,12 @@ private: if (available < to_write) { - throw Exception("Can not write past end of buffer. Space available " - + std::to_string(available) + " bytes, required to write: " - + std::to_string(to_write) + ".", - ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); + throw Exception(ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER, + "Can not write past end of buffer. Space available {} bytes, required to write {} bytes.", + available, to_write); } - const auto tmp_buffer = htobe64(static_cast(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8)); + const auto tmp_buffer = __builtin_bswap64(static_cast(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8)); memcpy(dest_current, &tmp_buffer, to_write); dest_current += to_write; From 0b3bf43d619ff4e753422a8cb0fbdff0d4815662 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 10 Sep 2021 23:56:24 +0300 Subject: [PATCH 60/89] Rewrite MergeTreeData::clearOldTemporaryDirectories() to use early continue --- src/Storages/MergeTree/MergeTreeData.cpp | 47 +++++++++++++----------- 1 file changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 790b95a9fa9..37e20204813 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1209,35 +1209,38 @@ void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifet { for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) { - if (startsWith(it->name(), "tmp_")) + const std::string & basename = it->name(); + if (!startsWith(basename, "tmp_")) { - try + continue; + } + + try + { + if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) { - if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) - { - LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); - disk->removeRecursive(it->path()); - } + LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); + disk->removeRecursive(it->path()); } - /// see getModificationTime() - catch (const ErrnoException & e) + } + /// see getModificationTime() + catch (const ErrnoException & e) + { + if (e.getErrno() == ENOENT) { - if (e.getErrno() == ENOENT) - { - /// If the file is already deleted, do nothing. - } - else - throw; + /// If the file is already deleted, do nothing. } - catch (const fs::filesystem_error & e) + else + throw; + } + catch (const fs::filesystem_error & e) + { + if (e.code() == std::errc::no_such_file_or_directory) { - if (e.code() == std::errc::no_such_file_or_directory) - { - /// If the file is already deleted, do nothing. - } - else - throw; + /// If the file is already deleted, do nothing. } + else + throw; } } } From 07e8b2b3c7bfe6181287607629002e8aa6bd354c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 11 Sep 2021 00:16:09 +0300 Subject: [PATCH 61/89] Do not try to remove temporary paths that is currently in written by merge/mutation v2: rebase against MergeTask v3: rebase due to conflicts in src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp v4: - rebase due to conflicts in src/Storages/MergeTree/MergeTask.cpp - drop common/scope_guard_safe.h (not used) --- src/Storages/MergeTree/MergeTask.cpp | 20 ++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++-- src/Storages/MergeTree/MergeTreeData.h | 3 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 ++++++ .../MergeTree/MergeTreeDataMergerMutator.h | 22 +++++++++++++++++++ .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 9 files changed, 63 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index aa3f91a4f00..2e123d849db 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -10,6 +10,7 @@ #include "Storages/MergeTree/IMergeTreeDataPart.h" #include "Storages/MergeTree/MergeTreeSequentialSource.h" #include "Storages/MergeTree/FutureMergedMutatedPart.h" +#include "Storages/MergeTree/MergeTreeDataMergerMutator.h" #include "Processors/Transforms/ExpressionTransform.h" #include "Processors/Transforms/MaterializingTransform.h" #include "Processors/Merges/MergingSortedTransform.h" @@ -117,11 +118,23 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } ctx->disk = global_ctx->space_reservation->getDisk(); - auto local_new_part_relative_tmp_path_name = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix; - auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_new_part_relative_tmp_path_name + "/"; + + String local_part_path = global_ctx->data->relative_data_path; + String local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + (global_ctx->parent_part ? ".proj" : ""); + String local_new_part_tmp_path = local_part_path + local_tmp_part_basename + "/"; + if (ctx->disk->exists(local_new_part_tmp_path)) throw Exception("Directory " + fullPath(ctx->disk, local_new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + { + std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); + global_ctx->mutator->tmp_parts.emplace(local_tmp_part_basename); + } + SCOPE_EXIT( + std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); + global_ctx->mutator->tmp_parts.erase(local_tmp_part_basename); + ); + global_ctx->all_column_names = global_ctx->metadata_snapshot->getColumns().getNamesOfPhysical(); global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); @@ -142,7 +155,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->future_part->type, global_ctx->future_part->part_info, local_single_disk_volume, - local_new_part_relative_tmp_path_name, + local_tmp_part_basename, global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; @@ -561,6 +574,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->new_data_part.get(), ".proj", global_ctx->data, + global_ctx->mutator, global_ctx->merges_blocker, global_ctx->ttl_merges_blocker)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 22dc70bd78c..80c8e7165f8 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -60,6 +60,7 @@ public: const IMergeTreeDataPart * parent_part_, String suffix_, MergeTreeData * data_, + MergeTreeDataMergerMutator * mutator_, ActionBlocker * merges_blocker_, ActionBlocker * ttl_merges_blocker_) { @@ -78,6 +79,7 @@ public: global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_); global_ctx->parent_part = std::move(parent_part_); global_ctx->data = std::move(data_); + global_ctx->mutator = std::move(mutator_); global_ctx->merges_blocker = std::move(merges_blocker_); global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_); @@ -121,6 +123,7 @@ private: std::unique_ptr projection_merge_list_element; MergeListElement * merge_list_element_ptr{nullptr}; MergeTreeData * data{nullptr}; + MergeTreeDataMergerMutator * mutator{nullptr}; ActionBlocker * merges_blocker{nullptr}; ActionBlocker * ttl_merges_blocker{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 37e20204813..83714b814a0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1193,7 +1193,7 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa } -void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds) +void MergeTreeData::clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds) { /// If the method is already called from another thread, then we don't need to do anything. std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock); @@ -1214,12 +1214,18 @@ void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifet { continue; } + const std::string & full_path = fullPath(disk, it->path()); + if (merger_mutator.hasTemporaryPart(basename)) + { + LOG_WARNING(log, "{} is an active destination for one of merge/mutation (consider increasing temporary_directories_lifetime setting)", full_path); + continue; + } try { if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) { - LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); + LOG_WARNING(log, "Removing temporary directory {}", full_path); disk->removeRecursive(it->path()); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e7f1db8f3ec..a05a106386c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -39,6 +39,7 @@ namespace DB class AlterCommands; class MergeTreePartsMover; +class MergeTreeDataMergerMutator; class MutationCommands; class Context; struct JobAndPool; @@ -536,7 +537,7 @@ public: /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. - void clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds); + void clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds); void clearEmptyParts(); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 5d97c64b49b..f4c23293bf2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -444,6 +444,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( parent_part, suffix, &data, + this, &merges_blocker, &ttl_merges_blocker); } @@ -774,4 +775,10 @@ ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadat } +bool MergeTreeDataMergerMutator::hasTemporaryPart(const std::string & basename) const +{ + std::lock_guard lock(tmp_parts_lock); + return tmp_parts.contains(basename); +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 22650ac4eca..e5c8a4d8285 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -136,6 +137,7 @@ private: MergeTreeData::DataPartsVector selectAllPartsFromPartition(const String & partition_id); friend class MutateTask; + friend class MergeTask; /** Split mutation commands into two parts: * First part should be executed by mutations interpreter. @@ -190,6 +192,26 @@ private: ITTLMergeSelector::PartitionIdToTTLs next_recompress_ttl_merge_times_by_partition; /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale + +public: + /// Returns true if passed part name is active. + /// (is the destination for one of active mutation/merge). + /// + /// NOTE: that it accept basename (i.e. dirname), not the path, + /// since later requires canonical form. + bool hasTemporaryPart(const std::string & basename) const; + +private: + /// Set of active temporary paths that is used as the destination. + /// List of such paths is required to avoid trying to remove them during cleanup. + /// + /// NOTE: It is pretty short, so use STL is fine. + std::unordered_set tmp_parts; + /// Lock for "tmp_parts". + /// + /// NOTE: mutable is required to mark hasTemporaryPath() const + mutable std::mutex tmp_parts_lock; + }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 06856c73888..5731092f2a8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -62,7 +62,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() /// Both use relative_data_path which changes during rename, so we /// do it under share lock storage.clearOldWriteAheadLogs(); - storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); + storage.clearOldTemporaryDirectories(storage.merger_mutator, storage.getSettings()->temporary_directories_lifetime.totalSeconds()); } /// This is loose condition: no problem if we actually had lost leadership at this moment diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ab42da1dfa0..0ed906b10f8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -105,7 +105,7 @@ void StorageMergeTree::startup() /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately - clearOldTemporaryDirectories(0); + clearOldTemporaryDirectories(merger_mutator, 0); /// NOTE background task will also do the above cleanups periodically. time_after_previous_cleanup_parts.restart(); @@ -1063,7 +1063,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( [this, share_lock] () { - clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); + clearOldTemporaryDirectories(merger_mutator, getSettings()->temporary_directories_lifetime.totalSeconds()); return true; }, common_assignee_trigger, getStorageID())); scheduled = true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..9eb5ab7f800 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -479,7 +479,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } /// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart), /// don't allow to reinitialize them, delete each of them immediately. - clearOldTemporaryDirectories(0); + clearOldTemporaryDirectories(merger_mutator, 0); clearOldWriteAheadLogs(); } From 55116ae399fdd09bc1808cdeabdf4d665831a697 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 01:57:22 +0300 Subject: [PATCH 62/89] Fix error --- base/base/phdr_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/phdr_cache.cpp b/base/base/phdr_cache.cpp index d2388666f73..8ae10f6bf83 100644 --- a/base/base/phdr_cache.cpp +++ b/base/base/phdr_cache.cpp @@ -6,7 +6,7 @@ #include -#if defined(__linux__) && !defined(THREAD_SANITIZER) +#if defined(__linux__) && !defined(THREAD_SANITIZER) && !defined(USE_MUSL) #define USE_PHDR_CACHE 1 #endif From 4848a6f97fd2203f7b46ea505b51dc395c1b6484 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 02:07:13 +0300 Subject: [PATCH 63/89] Add CMakeLists --- CMakeLists.txt | 6 +----- cmake/linux/default_libs.cmake | 10 +++++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 685b2c25a0d..2699132f165 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -188,7 +188,7 @@ endif () option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) elseif(GLIBC_COMPATIBILITY) @@ -203,10 +203,6 @@ if (GLIBC_COMPATIBILITY) set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h") endif() -if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") - message (WARNING "CMake version must be greater than 3.9.0 for production builds.") -endif () - # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index a2da7ba1915..4abd0a951e1 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -14,6 +14,8 @@ endif () if (OS_ANDROID) # pthread and rt are included in libc set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl") +elseif (USE_MUSL) + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc") else () set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl") endif () @@ -26,7 +28,7 @@ set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS}) # glibc-compatibility library relies to constant version of libc headers # (because minor changes in function attributes between different glibc versions will introduce incompatibilities) # This is for x86_64. For other architectures we have separate toolchains. -if (ARCH_AMD64 AND NOT_UNBUNDLED) +if (ARCH_AMD64 AND NOT_UNBUNDLED AND NOT CMAKE_CROSSCOMPILING) set(CMAKE_C_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers) set(CMAKE_CXX_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers) endif () @@ -37,8 +39,10 @@ set(THREADS_PREFER_PTHREAD_FLAG ON) find_package(Threads REQUIRED) if (NOT OS_ANDROID) - # Our compatibility layer doesn't build under Android, many errors in musl. - add_subdirectory(base/glibc-compatibility) + if (NOT USE_MUSL) + # Our compatibility layer doesn't build under Android, many errors in musl. + add_subdirectory(base/glibc-compatibility) + endif () add_subdirectory(base/harmful) endif () From 2c3709d8ae287d5b3739a1760a0298f8079df050 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 02:07:36 +0300 Subject: [PATCH 64/89] Add toolchain file --- cmake/linux/toolchain-x86_64-musl.cmake | 35 +++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 cmake/linux/toolchain-x86_64-musl.cmake diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake new file mode 100644 index 00000000000..b616a92bb06 --- /dev/null +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -0,0 +1,35 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "x86_64") +set (CMAKE_C_COMPILER_TARGET "x86_64-linux-musl") +set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-musl") +set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-musl") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl/x86_64-linux-musl-cross") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-musl") + +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (USE_MUSL 1) +add_definitions(-DUSE_MUSL=1) From 438d04795e851a29bacfb4f129bed4126aba7275 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 03:03:46 +0300 Subject: [PATCH 65/89] Update toolchain --- cmake/linux/toolchain-x86_64-musl.cmake | 4 ++-- contrib/sysroot | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake index b616a92bb06..0406b5de0ba 100644 --- a/cmake/linux/toolchain-x86_64-musl.cmake +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -6,9 +6,9 @@ set (CMAKE_C_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-musl") -set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl/x86_64-linux-musl-cross") +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl") -set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-musl") +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") diff --git a/contrib/sysroot b/contrib/sysroot index 002415524b5..e4663925b73 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 002415524b5d14124bb8a61a3ce7ac65774f5479 +Subproject commit e4663925b73beb57dd29154844c8d50441146753 From e7d99c3e544c209537dbc4ee93d65ccada7e22fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 03:09:38 +0300 Subject: [PATCH 66/89] Update submodule --- contrib/fastops | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/fastops b/contrib/fastops index 012b777df9e..1460583af7d 160000 --- a/contrib/fastops +++ b/contrib/fastops @@ -1 +1 @@ -Subproject commit 012b777df9e2d145a24800a6c8c3d4a0249bb09e +Subproject commit 1460583af7d13c0e980ce46aec8ee9400314669a From c18d2834690f1fc9ca5af8727e44823b08cd4631 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Oct 2021 03:57:26 +0300 Subject: [PATCH 67/89] Update codegen_select_fuzzer.cpp --- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 6a7a88a8545..9310d7d59f7 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -25,7 +25,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); - try + try { DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); From bf35af1b5d02091f34d2e85387411102140de7c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 16 Oct 2021 09:40:22 +0300 Subject: [PATCH 68/89] Fix ProfileInfo. --- src/QueryPipeline/ProfileInfo.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index 32986f7259c..ee0ff8c69bf 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -46,12 +46,14 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) size_t ProfileInfo::getRowsBeforeLimit() const { + calculated_rows_before_limit = true; return rows_before_limit; } bool ProfileInfo::hasAppliedLimit() const { + calculated_rows_before_limit = true; return applied_limit; } From 6c0eaf76da0a583c8046e7d9c6593f44dc2a8c97 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 11:41:50 +0300 Subject: [PATCH 69/89] Query stage for local --- src/Client/LocalConnection.cpp | 13 ++++++----- src/Client/LocalConnection.h | 2 +- .../02048_clickhouse_local_stage.reference | 15 +++++++++++++ .../02048_clickhouse_local_stage.sh | 22 +++++++++++++++++++ 4 files changed, 45 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02048_clickhouse_local_stage.reference create mode 100755 tests/queries/0_stateless/02048_clickhouse_local_stage.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index efd302622dd..4455ba3b9ad 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -60,15 +60,15 @@ void LocalConnection::updateProgress(const Progress & value) void LocalConnection::sendQuery( const ConnectionTimeouts &, - const String & query_, - const String & query_id_, - UInt64, + const String & query, + const String & query_id, + UInt64 stage, const Settings *, const ClientInfo *, bool) { query_context = session.makeQueryContext(); - query_context->setCurrentQueryId(query_id_); + query_context->setCurrentQueryId(query_id); if (send_progress) query_context->setProgressCallback([this] (const Progress & value) { return this->updateProgress(value); }); @@ -77,8 +77,9 @@ void LocalConnection::sendQuery( state.reset(); state.emplace(); - state->query_id = query_id_; - state->query = query_; + state->query_id = query_id; + state->query = query; + state->stage = QueryProcessingStage::Enum(stage); if (send_progress) state->after_send_progress.restart(); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index dcea3ed0fc3..242d23ddc36 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -76,7 +76,7 @@ public: void sendQuery( const ConnectionTimeouts & timeouts, const String & query, - const String & query_id_/* = "" */, + const String & query_id/* = "" */, UInt64 stage/* = QueryProcessingStage::Complete */, const Settings * settings/* = nullptr */, const ClientInfo * client_info/* = nullptr */, diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference new file mode 100644 index 00000000000..44c39f2a444 --- /dev/null +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -0,0 +1,15 @@ +execute: default +"foo" +1 +execute: --stage fetch_columns +"dummy" +0 +execute: --stage with_mergeable_state +"1" +1 +execute: --stage with_mergeable_state_after_aggregation +"1" +1 +execute: --stage complete +"foo" +1 diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh new file mode 100755 index 00000000000..5c1303b5160 --- /dev/null +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function execute_query() +{ + if [ $# -eq 0 ]; then + echo "execute: default" + else + echo "execute: $*" + fi + ${CLICKHOUSE_LOCAL} "$@" --format CSVWithNames -q "SELECT 1 AS foo" +} + +execute_query # default -- complete +execute_query --stage fetch_columns +execute_query --stage with_mergeable_state +execute_query --stage with_mergeable_state_after_aggregation +execute_query --stage complete From 50231460af503e6e723155fa893e21de1b36f7e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 11:28:10 +0300 Subject: [PATCH 70/89] Use forward declaration for Buffer<> in generic headers - changes in ReadHelpers.h -- recompiles 1000 modules - changes in FormatFactor.h -- recompiles 100 modules --- src/Dictionaries/SSDCacheDictionaryStorage.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 7 +++++-- src/Formats/JSONEachRowUtils.cpp | 1 + src/Formats/JSONEachRowUtils.h | 4 ++++ src/IO/ReadHelpers.cpp | 5 +++-- src/IO/ReadHelpers.h | 9 ++++++--- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 + .../Formats/Impl/TabSeparatedRowInputFormat.cpp | 1 + 9 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index ffe0694d8c4..7c53ecc2b2c 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 34574ca13f8..ec7fa0a9e80 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -13,6 +13,7 @@ #include #include +#include #include namespace DB diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index f20cec56943..d5784219c6a 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include #include -#include #include #include @@ -34,6 +34,9 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; +template +struct Memory; + FormatSettings getFormatSettings(ContextPtr context); template @@ -55,7 +58,7 @@ public: */ using FileSegmentationEngine = std::function( ReadBuffer & buf, - DB::Memory<> & memory, + DB::Memory> & memory, size_t min_chunk_bytes)>; /// This callback allows to perform some additional actions after writing a single row. diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index b918825df79..8ef05fa584e 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 79dd6c6c192..2d2d4ad5531 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -1,5 +1,9 @@ #pragma once +#include +#include +#include + namespace DB { diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ffa050b71c8..5fe0fda88cd 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -1120,7 +1121,7 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } -void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) +void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current) { assert(current >= in.position()); assert(current <= in.buffer().end()); @@ -1140,7 +1141,7 @@ void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) in.position() = current; } -bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) +bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current) { assert(current <= in.buffer().end()); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ca6affbf907..880173bfd96 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -29,7 +30,6 @@ #include #include #include -#include #include #include @@ -41,6 +41,9 @@ static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB; namespace DB { +template +struct Memory; + namespace ErrorCodes { extern const int CANNOT_PARSE_DATE; @@ -1290,7 +1293,7 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); /** This function just copies the data from buffer's internal position (in.position()) * to current position (from arguments) into memory. */ -void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); +void saveUpToPosition(ReadBuffer & in, Memory> & memory, char * current); /** This function is negative to eof(). * In fact it returns whether the data was loaded to internal ReadBuffers's buffer or not. @@ -1299,7 +1302,7 @@ void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); * of our buffer and the current cursor in the end of the buffer. When we call eof() it calls next(). * And this function can fill the buffer with new data, so we will lose the data from previous buffer state. */ -bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current); +bool loadAtPosition(ReadBuffer & in, Memory> & memory, char * & current); struct PcgDeserializer diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8ccc04faf35..4beb260b64a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 1ff52c9f695..c1cf0a904ea 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include From 77a2022cf543226e2116c6ca896f7b226c0f9364 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 9 Oct 2021 17:00:39 +0800 Subject: [PATCH 71/89] Add primary key to minmax_count_projection --- src/Storages/MergeTree/MergeTreeData.cpp | 120 +++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +- .../MergeTree/registerStorageMergeTree.cpp | 18 +-- src/Storages/ProjectionsDescription.cpp | 19 ++- src/Storages/ProjectionsDescription.h | 6 +- .../01710_minmax_count_projection.reference | 5 + .../01710_minmax_count_projection.sql | 28 ++++ 8 files changed, 157 insertions(+), 44 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c5b5e2a9a92..66b7f62a7c8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4398,6 +4398,8 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, const SelectQueryInfo & query_info, + const DataPartsVector & parts, + DataPartsVector & normal_parts, ContextPtr query_context) const { if (!metadata_snapshot->minmax_count_projection) @@ -4406,7 +4408,14 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( ErrorCodes::LOGICAL_ERROR); auto block = metadata_snapshot->minmax_count_projection->sample_block; + String primary_key_max_column_name; + if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + primary_key_max_column_name = *(block.getNames().cend() - 2); + bool need_primary_key_max_column = std::any_of( + required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + auto minmax_count_columns = block.mutateColumns(); + auto minmax_count_columns_size = minmax_count_columns.size(); auto insert = [](ColumnAggregateFunction & column, const Field & value) { @@ -4422,7 +4431,6 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( column.insertFrom(place); }; - auto parts = getDataPartsVector(); ASTPtr expression_ast; Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */); if (virtual_columns_block.rows() == 0) @@ -4446,15 +4454,13 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( if (!part->minmax_idx->initialized) throw Exception("Found a non-empty part with uninitialized minmax_idx. It's a bug", ErrorCodes::LOGICAL_ERROR); - size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); - if (2 * minmax_idx_size + 1 != minmax_count_columns.size()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "minmax_count projection should have twice plus one the number of ranges in minmax_idx. 2 * minmax_idx_size + 1 = {}, " - "minmax_count_columns.size() = {}. It's a bug", - 2 * minmax_idx_size + 1, - minmax_count_columns.size()); + if (need_primary_key_max_column && !part->index_granularity.hasFinalMark()) + { + normal_parts.push_back(part); + continue; + } + size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); for (size_t i = 0; i < minmax_idx_size; ++i) { size_t min_pos = i * 2; @@ -4466,6 +4472,16 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( insert(max_column, range.right); } + if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + { + const auto & primary_key_column = *part->index[0]; + auto primary_key_column_size = primary_key_column.size(); + auto & min_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 3]); + auto & max_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 2]); + insert(min_column, primary_key_column[0]); + insert(max_column, primary_key_column[primary_key_column_size - 1]); + } + { auto & column = assert_cast(*minmax_count_columns.back()); auto func = column.getAggregateFunction(); @@ -4722,33 +4738,74 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( size_t min_sum_marks = std::numeric_limits::max(); if (metadata_snapshot->minmax_count_projection) add_projection_candidate(*metadata_snapshot->minmax_count_projection); + std::optional minmax_conut_projection_candidate; + if (!candidates.empty()) + { + minmax_conut_projection_candidate.emplace(std::move(candidates.front())); + candidates.clear(); + } + MergeTreeDataSelectExecutor reader(*this); + std::shared_ptr max_added_blocks; + if (settings.select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + auto parts = getDataPartsVector(); - // Only add more projection candidates if minmax_count_projection cannot match. - if (candidates.empty()) + // If minmax_count_projection is a valid candidate, check its completeness. + if (minmax_conut_projection_candidate) + { + DataPartsVector normal_parts; + query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock( + metadata_snapshot, minmax_conut_projection_candidate->required_columns, query_info, parts, normal_parts, query_context); + + if (normal_parts.empty()) + { + selected_candidate = &*minmax_conut_projection_candidate; + selected_candidate->complete = true; + min_sum_marks = query_info.minmax_count_projection_block.rows(); + } + else + { + if (normal_parts.size() == parts.size()) + { + // minmax_count_projection is useless. + } + else + { + auto normal_result_ptr = reader.estimateNumMarksToRead( + normal_parts, + analysis_result.required_columns, + metadata_snapshot, + metadata_snapshot, + query_info, + query_context, + settings.max_threads, + max_added_blocks); + + if (!normal_result_ptr->error()) + { + selected_candidate = &*minmax_conut_projection_candidate; + selected_candidate->merge_tree_normal_select_result_ptr = normal_result_ptr; + min_sum_marks = query_info.minmax_count_projection_block.rows() + normal_result_ptr->marks(); + } + } + + // We cannot find a complete match of minmax_count_projection, add more projections to check. + for (const auto & projection : metadata_snapshot->projections) + add_projection_candidate(projection); + } + } + else { for (const auto & projection : metadata_snapshot->projections) add_projection_candidate(projection); } - else - { - selected_candidate = &candidates.front(); - query_info.minmax_count_projection_block - = getMinMaxCountProjectionBlock(metadata_snapshot, selected_candidate->required_columns, query_info, query_context); - min_sum_marks = query_info.minmax_count_projection_block.rows(); - } // Let's select the best projection to execute the query. - if (!candidates.empty() && !selected_candidate) + if (!candidates.empty()) { - std::shared_ptr max_added_blocks; - if (settings.select_sequential_consistency) - { - if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) - max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); - } - - auto parts = getDataPartsVector(); - MergeTreeDataSelectExecutor reader(*this); query_info.merge_tree_select_result_ptr = reader.estimateNumMarksToRead( parts, analysis_result.required_columns, @@ -4763,7 +4820,12 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( { // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. - min_sum_marks = query_info.merge_tree_select_result_ptr->marks() + 1; + auto sum_marks = query_info.merge_tree_select_result_ptr->marks() + 1; + if (sum_marks < min_sum_marks) + { + selected_candidate = nullptr; + min_sum_marks = sum_marks; + } } /// Favor aggregate projections diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b87a756bf9f..009abbb7b3b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -368,6 +368,8 @@ public: const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, const SelectQueryInfo & query_info, + const DataPartsVector & parts, + DataPartsVector & normal_parts, ContextPtr query_context) const; bool getQueryProcessingStageWithAggregateProjection( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a33296cbf24..55de92735c3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -162,7 +162,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( LOG_DEBUG( log, - "Choose {} projection {}", + "Choose {} {} projection {}", + query_info.projection->complete ? "complete" : "incomplete", query_info.projection->desc->type, query_info.projection->desc->name); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 379573a381f..982acfe62a4 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -651,10 +651,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// single default partition with name "all". metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, args.getContext()); - auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, args.getContext())); - /// PRIMARY KEY without ORDER BY is allowed and considered as ORDER BY. if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); @@ -686,6 +682,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.primary_key.definition_ast = nullptr; } + auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); + auto primary_key_asts = metadata.primary_key.expression_list_ast->children; + metadata.minmax_count_projection.emplace( + ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.getContext()); @@ -736,10 +737,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata.columns, args.getContext()); - auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, args.getContext())); - ++arg_num; /// If there is an expression for sampling @@ -765,6 +762,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) ++arg_num; + auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); + auto primary_key_asts = metadata.primary_key.expression_list_ast->children; + metadata.minmax_count_projection.emplace( + ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) storage_settings->index_granularity = safeGet(ast->value); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 42294b8152c..5ad79304e27 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -60,6 +60,7 @@ ProjectionDescription ProjectionDescription::clone() const other.metadata = metadata; other.key_size = key_size; other.is_minmax_count_projection = is_minmax_count_projection; + other.has_primary_key_minmax = has_primary_key_minmax; return other; } @@ -172,9 +173,15 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const return result; } -ProjectionDescription -ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & columns, const Names & minmax_columns, ContextPtr query_context) +ProjectionDescription ProjectionDescription::getMinMaxCountProjection( + const ColumnsDescription & columns, + const Names & minmax_columns, + const ASTs & primary_key_asts, + ContextPtr query_context) { + ProjectionDescription result; + result.is_minmax_count_projection = true; + auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); for (const auto & column : minmax_columns) @@ -182,10 +189,15 @@ ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & colum select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); } + if (!primary_key_asts.empty()) + { + select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); + select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); + result.has_primary_key_minmax = true; + } select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); - ProjectionDescription result; result.definition_ast = select_query; result.name = MINMAX_COUNT_PROJECTION_NAME; result.query_ast = select_query->cloneToASTSelect(); @@ -203,7 +215,6 @@ ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & colum metadata.sorting_key = KeyDescription::buildEmptyKey(); metadata.primary_key = KeyDescription::buildEmptyKey(); result.metadata = std::make_shared(metadata); - result.is_minmax_count_projection = true; return result; } diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index b9c11cb0771..28cff7ce532 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -58,12 +58,14 @@ struct ProjectionDescription bool is_minmax_count_projection = false; + bool has_primary_key_minmax = false; + /// Parse projection from definition AST static ProjectionDescription getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context); - static ProjectionDescription - getMinMaxCountProjection(const ColumnsDescription & columns, const Names & minmax_columns, ContextPtr query_context); + static ProjectionDescription getMinMaxCountProjection( + const ColumnsDescription & columns, const Names & minmax_columns, const ASTs & primary_key_asts, ContextPtr query_context); ProjectionDescription() = default; diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index ad9b87b998d..5591d5a9954 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -3,3 +3,8 @@ 1 9999 5000 0 9998 5000 1 +0 +0 +0 +0 9999 +0 9999 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 58af11f01f7..112487b219e 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -15,3 +15,31 @@ select min(i), max(i), count() from d where _partition_value.1 = 10 group by _pa select min(i) from d where 1 = _partition_value.1; drop table d; + +drop table if exists no_final_mark; +drop table if exists has_final_mark; +drop table if exists mixed_final_mark; + +create table no_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10, write_final_mark = 0; +create table has_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10, write_final_mark = 1; +create table mixed_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10; + +set max_rows_to_read = 100000; + +insert into no_final_mark select number, number from numbers(10000); +insert into has_final_mark select number, number from numbers(10000); + +alter table mixed_final_mark attach partition 0 from no_final_mark; +alter table mixed_final_mark attach partition 1 from has_final_mark; + +set max_rows_to_read = 2; + +select min(j) from no_final_mark; +select min(j) from has_final_mark; +select min(j) from mixed_final_mark; + +select min(j), max(j) from no_final_mark; -- {serverError TOO_MANY_ROWS} +select min(j), max(j) from has_final_mark; + +set max_rows_to_read = 5001; -- one normal part 5000 + one minmax_count_projection part 1 +select min(j), max(j) from mixed_final_mark; From 77c4a5fa18ea84a33d65d2e67b5bf7d5b9c67263 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 14 Oct 2021 01:31:37 +0800 Subject: [PATCH 72/89] Better --- src/Storages/MergeTree/MergeTreeData.cpp | 38 ++++++++++++------------ src/Storages/MergeTree/MergeTreeData.h | 10 +++++++ src/Storages/ProjectionsDescription.cpp | 10 +++---- src/Storages/ProjectionsDescription.h | 5 ++++ 4 files changed, 39 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 66b7f62a7c8..1f1b9a3f0a8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4408,15 +4408,16 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( ErrorCodes::LOGICAL_ERROR); auto block = metadata_snapshot->minmax_count_projection->sample_block; + bool need_primary_key_max_column = false; String primary_key_max_column_name; if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) - primary_key_max_column_name = *(block.getNames().cend() - 2); - bool need_primary_key_max_column = std::any_of( - required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + { + primary_key_max_column_name = block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; + need_primary_key_max_column = std::any_of( + required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + } auto minmax_count_columns = block.mutateColumns(); - auto minmax_count_columns_size = minmax_count_columns.size(); - auto insert = [](ColumnAggregateFunction & column, const Field & value) { auto func = column.getAggregateFunction(); @@ -4460,28 +4461,27 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( continue; } - size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); - for (size_t i = 0; i < minmax_idx_size; ++i) - { - size_t min_pos = i * 2; - size_t max_pos = i * 2 + 1; - auto & min_column = assert_cast(*minmax_count_columns[min_pos]); - auto & max_column = assert_cast(*minmax_count_columns[max_pos]); - const auto & range = part->minmax_idx->hyperrectangle[i]; - insert(min_column, range.left); - insert(max_column, range.right); - } - + size_t pos = 0; if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) { const auto & primary_key_column = *part->index[0]; auto primary_key_column_size = primary_key_column.size(); - auto & min_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 3]); - auto & max_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 2]); + auto & min_column = assert_cast(*minmax_count_columns[pos++]); + auto & max_column = assert_cast(*minmax_count_columns[pos++]); insert(min_column, primary_key_column[0]); insert(max_column, primary_key_column[primary_key_column_size - 1]); } + size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); + for (size_t i = 0; i < minmax_idx_size; ++i) + { + auto & min_column = assert_cast(*minmax_count_columns[pos++]); + auto & max_column = assert_cast(*minmax_count_columns[pos++]); + const auto & range = part->minmax_idx->hyperrectangle[i]; + insert(min_column, range.left); + insert(max_column, range.right); + } + { auto & column = assert_cast(*minmax_count_columns.back()); auto func = column.getAggregateFunction(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 009abbb7b3b..7151141cb15 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -364,6 +364,16 @@ public: bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); + /// Build a block of minmax and count values of a MergeTree table. These values are extracted + /// from minmax_indices, the first expression of primary key, and part rows. + /// + /// query_info - used to filter unneeded parts + /// + /// parts - part set to filter + /// + /// normal_parts - collects parts that don't have all the needed values to form the block. + /// Specifically, this is when a part doesn't contain a final mark and the related max value is + /// required. Block getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 5ad79304e27..7b27604730c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -184,17 +184,17 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); - for (const auto & column : minmax_columns) - { - select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); - select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); - } if (!primary_key_asts.empty()) { select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); result.has_primary_key_minmax = true; } + for (const auto & column : minmax_columns) + { + select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); + select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); + } select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 28cff7ce532..77b858b3ab1 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -30,6 +30,10 @@ struct ProjectionDescription static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection"; + /// If minmax_count projection contains a primary key's minmax values. Their positions will be 0 and 1. + static constexpr const size_t PRIMARY_KEY_MIN_COLUMN_POS = 0; + static constexpr const size_t PRIMARY_KEY_MAX_COLUMN_POS = 1; + /// Definition AST of projection ASTPtr definition_ast; @@ -58,6 +62,7 @@ struct ProjectionDescription bool is_minmax_count_projection = false; + /// If a primary key expression is used in the minmax_count projection, this flag will be true. bool has_primary_key_minmax = false; /// Parse projection from definition AST From 9ceb668a4ddb614291ae74bab2b4b582bf6750ec Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 16 Oct 2021 18:13:11 +0800 Subject: [PATCH 73/89] Fix TSan --- src/Storages/MergeTree/MergeTreeData.cpp | 9 ++++----- src/Storages/ProjectionsDescription.cpp | 5 +++-- src/Storages/ProjectionsDescription.h | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1f1b9a3f0a8..45748539e9d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4407,12 +4407,11 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( "Cannot find the definition of minmax_count projection but it's used in current query. It's a bug", ErrorCodes::LOGICAL_ERROR); - auto block = metadata_snapshot->minmax_count_projection->sample_block; + auto block = metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); bool need_primary_key_max_column = false; - String primary_key_max_column_name; - if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name; + if (!primary_key_max_column_name.empty()) { - primary_key_max_column_name = block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; need_primary_key_max_column = std::any_of( required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); } @@ -4462,7 +4461,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; - if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + if (!primary_key_max_column_name.empty()) { const auto & primary_key_column = *part->index[0]; auto primary_key_column_size = primary_key_column.size(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 7b27604730c..b3b344e9ecc 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -60,7 +60,7 @@ ProjectionDescription ProjectionDescription::clone() const other.metadata = metadata; other.key_size = key_size; other.is_minmax_count_projection = is_minmax_count_projection; - other.has_primary_key_minmax = has_primary_key_minmax; + other.primary_key_max_column_name = primary_key_max_column_name; return other; } @@ -188,7 +188,6 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( { select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); - result.has_primary_key_minmax = true; } for (const auto & column : minmax_columns) { @@ -208,6 +207,8 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); + if (!primary_key_asts.empty()) + result.primary_key_max_column_name = result.sample_block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; result.type = ProjectionDescription::Type::Aggregate; StorageInMemoryMetadata metadata; metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList())); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 77b858b3ab1..4dd717239ad 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -62,8 +62,8 @@ struct ProjectionDescription bool is_minmax_count_projection = false; - /// If a primary key expression is used in the minmax_count projection, this flag will be true. - bool has_primary_key_minmax = false; + /// If a primary key expression is used in the minmax_count projection, store the name of max expression. + String primary_key_max_column_name; /// Parse projection from definition AST static ProjectionDescription From 1b50d26c5122df276379e4cd5a2d86aa456c65de Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 16 Oct 2021 13:41:56 +0300 Subject: [PATCH 74/89] Added documentation --- .../external-dicts-dict-layout.md | 54 +++++++++++++++++++ src/Dictionaries/HashedArrayDictionary.cpp | 11 ++-- 2 files changed, 60 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index f525ea64aa2..30e050ef9ef 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -155,6 +155,60 @@ Configuration example: LAYOUT(COMPLEX_KEY_HASHED()) ``` +### complex_key_sparse_hashed {#complex-key-sparse-hashed} + +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `sparse_hashed`. + +Configuration example: + +``` xml + + + +``` + +``` sql +LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) +``` + +### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} + +The dictionary is completely stored in memory. Each attribute is stored in array. Key attribute is stored in the form of hashed table where value is index in attributes array. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. + +All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. + +Configuration example: + +``` xml + + + + +``` + +or + +``` sql +LAYOUT(HASHED_ARRAY()) +``` + +### complex_key_hashed_array {#complex-key-hashed-array} + +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed_array`. + +Configuration example: + +``` xml + + + +``` + +``` sql +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +``` + + ### range_hashed {#range-hashed} The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 7706f7d6108..3c02f377c3e 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -622,6 +622,9 @@ void HashedArrayDictionary::calculateBytesAllocated() if (attribute.string_arena) bytes_allocated += attribute.string_arena->size(); + + if (attribute.is_index_null.has_value()) + bytes_allocated += (*attribute.is_index_null).size(); } bytes_allocated += complex_key_arena.size(); @@ -634,14 +637,12 @@ template Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size) const { PaddedPODArray keys; + keys.reserve(key_attribute.container.size()); - for (auto & [key, value] : key_attribute.container) + for (auto & [key, _] : key_attribute.container) keys.emplace_back(key); - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); - else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template class HashedArrayDictionary; From c79c0526ff5f9783648a85e6c87657ecb9d2d060 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 15:18:00 +0300 Subject: [PATCH 75/89] Fix clickhouse-local syntax exception --- programs/local/LocalServer.cpp | 6 ++---- .../02050_clickhouse_local_parsing_exception.reference | 1 + .../02050_clickhouse_local_parsing_exception.sh | 8 ++++++++ 3 files changed, 11 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference create mode 100755 tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 30082caaac1..10c4bf592ca 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -128,9 +127,8 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) } case MultiQueryProcessingStage::PARSING_EXCEPTION: { - this_query_end = find_first_symbols<'\n'>(this_query_end, all_queries_end); - this_query_begin = this_query_end; /// It's expected syntax error, skip the line - current_exception.reset(); + if (current_exception) + current_exception->rethrow(); continue; } case MultiQueryProcessingStage::EXECUTE_QUERY: diff --git a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh new file mode 100755 index 00000000000..7a92fa6fefe --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native. Expected string literal." && echo 'OK' || echo 'FAIL' ||: + From a08c98d7603e6207ae4a7af2bd78c543602fa76e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 16 Oct 2021 17:03:50 +0300 Subject: [PATCH 76/89] Move some files. --- programs/copier/ClusterCopier.cpp | 4 ++-- programs/library-bridge/Handlers.cpp | 4 ++-- programs/obfuscator/Obfuscator.cpp | 4 ++-- programs/odbc-bridge/MainHandler.cpp | 2 +- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- src/Client/Connection.cpp | 4 ++-- src/Client/IServerConnection.h | 2 +- src/Core/ExternalTable.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- src/Dictionaries/DictionaryHelpers.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 2 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/IDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 ++-- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Dictionaries/readInvalidateQuery.cpp | 2 +- src/Formats/TemporaryFileStream.h | 2 +- src/Formats/formatBlock.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 2 +- src/Interpreters/SortedBlocksWriter.h | 2 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Processors/Executors/CompletedPipelineExecutor.cpp | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 2 +- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PullingPipelineExecutor.cpp | 2 +- src/Processors/Executors/PushingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PushingPipelineExecutor.cpp | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 2 +- src/Processors/QueryPlan/ArrayJoinStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 +- src/Processors/QueryPlan/CubeStep.cpp | 2 +- src/Processors/QueryPlan/DistinctStep.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 2 +- src/Processors/QueryPlan/ExtremesStep.cpp | 2 +- src/Processors/QueryPlan/FillingStep.cpp | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 2 +- src/Processors/QueryPlan/FinishSortingStep.cpp | 2 +- src/Processors/QueryPlan/ISourceStep.cpp | 2 +- src/Processors/QueryPlan/ITransformingStep.cpp | 2 +- src/Processors/QueryPlan/IntersectOrExceptStep.cpp | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 2 +- src/Processors/QueryPlan/LimitByStep.cpp | 2 +- src/Processors/QueryPlan/LimitStep.cpp | 2 +- src/Processors/QueryPlan/MergeSortingStep.cpp | 2 +- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 2 +- src/Processors/QueryPlan/MergingSortedStep.cpp | 2 +- src/Processors/QueryPlan/OffsetStep.cpp | 2 +- src/Processors/QueryPlan/PartialSortingStep.cpp | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Processors/QueryPlan/ReadFromPreparedSource.cpp | 2 +- src/Processors/QueryPlan/ReadFromPreparedSource.h | 2 +- src/Processors/QueryPlan/ReadNothingStep.cpp | 2 +- src/Processors/QueryPlan/RollupStep.cpp | 2 +- src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp | 2 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 2 +- src/Processors/QueryPlan/UnionStep.cpp | 2 +- src/Processors/QueryPlan/WindowStep.cpp | 2 +- src/Processors/Sources/DelayedSource.h | 2 +- src/Processors/Sources/RemoteSource.h | 2 +- src/Processors/Sources/ShellCommandSource.h | 2 +- src/Processors/Transforms/AggregatingTransform.cpp | 2 +- src/Processors/Transforms/CreatingSetsTransform.h | 4 ++-- .../MergingAggregatedMemoryEfficientTransform.cpp | 2 +- src/Processors/Transforms/buildPushingToViewsChain.h | 2 +- src/Processors/Transforms/getSourceFromASTInsertQuery.cpp | 2 +- src/QueryPipeline/BlockIO.h | 2 +- src/{Processors => QueryPipeline}/Chain.cpp | 2 +- src/{Processors => QueryPipeline}/Chain.h | 2 +- src/{Processors => QueryPipeline}/Pipe.cpp | 2 +- src/{Processors => QueryPipeline}/Pipe.h | 4 ++-- .../PipelineResourcesHolder.cpp | 2 +- src/{Processors => QueryPipeline}/PipelineResourcesHolder.h | 0 src/{Processors => QueryPipeline}/QueryPipeline.cpp | 6 +++--- src/{Processors => QueryPipeline}/QueryPipeline.h | 2 +- src/{Processors => QueryPipeline}/QueryPipelineBuilder.cpp | 2 +- src/{Processors => QueryPipeline}/QueryPipelineBuilder.h | 4 ++-- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- src/QueryPipeline/narrowBlockInputStreams.cpp | 2 +- src/{Processors => QueryPipeline}/printPipeline.cpp | 2 +- src/{Processors => QueryPipeline}/printPipeline.h | 0 .../tests/gtest_blocks_size_merging_streams.cpp | 4 ++-- src/QueryPipeline/tests/gtest_check_sorted_stream.cpp | 4 ++-- src/Server/GRPCServer.cpp | 2 +- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 4 ++-- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 2 +- src/Storages/LiveView/StorageBlocks.h | 2 +- src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/PartitionCommands.cpp | 2 +- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 2 +- src/Storages/ReadFinalForExternalReplicaStorage.h | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/StorageDictionary.cpp | 2 +- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageInput.cpp | 2 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMaterializedMySQL.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageS3.cpp | 4 ++-- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageTableFunction.h | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageValues.cpp | 2 +- src/Storages/StorageView.cpp | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 2 +- src/Storages/System/StorageSystemOne.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 2 +- src/Storages/tests/gtest_storage_log.cpp | 4 ++-- 141 files changed, 154 insertions(+), 154 deletions(-) rename src/{Processors => QueryPipeline}/Chain.cpp (99%) rename src/{Processors => QueryPipeline}/Chain.h (97%) rename src/{Processors => QueryPipeline}/Pipe.cpp (99%) rename src/{Processors => QueryPipeline}/Pipe.h (98%) rename src/{Processors => QueryPipeline}/PipelineResourcesHolder.cpp (94%) rename src/{Processors => QueryPipeline}/PipelineResourcesHolder.h (100%) rename src/{Processors => QueryPipeline}/QueryPipeline.cpp (99%) rename src/{Processors => QueryPipeline}/QueryPipeline.h (98%) rename src/{Processors => QueryPipeline}/QueryPipelineBuilder.cpp (99%) rename src/{Processors => QueryPipeline}/QueryPipelineBuilder.h (99%) rename src/{Processors => QueryPipeline}/printPipeline.cpp (99%) rename src/{Processors => QueryPipeline}/printPipeline.h (100%) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index e7aeea8cbad..5b21a7c2aef 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index abc5118baad..bf9ace679ba 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -11,11 +11,11 @@ #include #include #include -#include +#include #include #include #include -#include +#include #include #include diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index f18281a9fdf..caccc726923 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include @@ -25,7 +25,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 6362c7dfa9b..82d1bd61c24 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 9bc14627ac3..e5c6c09ba62 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a226407e276..21d605e7b45 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -45,7 +45,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 5841c6136af..3c920e8cabe 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -25,8 +25,8 @@ #include "Core/Block.h" #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 7424afc969d..cb76c7ff639 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -8,7 +8,7 @@ #include -#include +#include #include #include diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 4dd8b0cf016..b4adbcc0662 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 83fb101dba8..71b0c2ec6ea 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -13,7 +13,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index ab144761e11..851c9bc74bd 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 8de42760dc2..0facdfc20be 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index d2c1195c0c5..86481d9fd84 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 7683f9d4244..f7e9ce6624c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include namespace ProfileEvents { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index a19eca1fee7..a5a04d277da 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 6266bd2cf4f..65c40898983 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -16,7 +16,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 03d3b579ec3..f4f33439638 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 532c68d0453..a7cf69bb14d 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/IDictionarySource.h b/src/Dictionaries/IDictionarySource.h index 661f5b8eeb8..5071b69d2bf 100644 --- a/src/Dictionaries/IDictionarySource.h +++ b/src/Dictionaries/IDictionarySource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 5fabe9cf287..f6de6ca0cc1 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -11,8 +11,8 @@ #include "registerDictionaries.h" #include #include -#include -#include +#include +#include #include diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 8d0f0f1abc4..0a9ba1f5ea8 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index a3324b7d769..c9fb8b86b77 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -11,7 +11,7 @@ #include #include "readInvalidateQuery.h" #include -#include +#include #include #endif diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index b0be90c26a5..ba993ec5783 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -16,7 +16,7 @@ #include "readInvalidateQuery.h" #include "registerDictionaries.h" #include -#include +#include #include diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index c2c42eece58..370e1457832 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,5 @@ #include "readInvalidateQuery.h" -#include +#include #include #include #include diff --git a/src/Formats/TemporaryFileStream.h b/src/Formats/TemporaryFileStream.h index 5a1e0bc870a..4a2aa2d55e0 100644 --- a/src/Formats/TemporaryFileStream.h +++ b/src/Formats/TemporaryFileStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Formats/formatBlock.cpp b/src/Formats/formatBlock.cpp index 3284663fc42..d2b401207aa 100644 --- a/src/Formats/formatBlock.cpp +++ b/src/Formats/formatBlock.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 7e0fa2ba003..cc04c5f013f 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 95b279fd59b..0db07267231 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 487fa2538c2..3afb1e1fbb0 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 59fd1009381..5dbde2344d3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -33,7 +33,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 7789c74d596..7fb9f1a3ceb 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3d0813579ce..5f08ae6b845 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index ebec58dcca7..1945824636f 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index ac58ef2ab7b..0262a274c68 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 45e1c580f01..c3b8cc5c677 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index be388386e9d..a4e3dea89fa 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 0616a392027..ec07cee8738 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 80ed8225c79..fdddfdef2a4 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index ad7da63b0b1..a9c73b9f8fb 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index a1a0755f952..0b6d5334716 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 2e2b5e9ca1e..c1e851e3425 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 023f9016cc4..53036360b8d 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index 35b974baa83..3ca5b9109e6 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 83a4c291bf2..45c3719ebca 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 3d61d3ef36b..23c5115ec68 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index d53d1fa9310..5aeb33fdc7b 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index d1b9c2cad63..33d2ad6e1cf 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ExtremesStep.cpp b/src/Processors/QueryPlan/ExtremesStep.cpp index 117ccd414ca..4524b9883d6 100644 --- a/src/Processors/QueryPlan/ExtremesStep.cpp +++ b/src/Processors/QueryPlan/ExtremesStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 204559ecc3b..223892aa528 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 483055810cf..df75c37dc97 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index c219c09f3bd..6347b69901c 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 61c0a9254cd..0644d9b44eb 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index 1c7f836378f..629fb89be1e 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index a4d81e69fe0..c031303cc7f 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 9c5f8ae2e5f..494a2a6aa0e 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index 12ad933a159..39086e995fc 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 3db59e0684a..8c5e3e3c87c 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp index 820bbc31b74..534f05a4d6e 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ b/src/Processors/QueryPlan/MergeSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index d02be59ae84..8dfb9f9c923 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 87d1af4d2bd..ed1f24db55b 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index b48327eb36c..e0c70ba2f28 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index cf7cb157e4c..420e7b583ca 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 6fb6a24f65b..f319e562bfb 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8d3005e725f..cc400aacf2a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index c8213d58db6..fc8136177cf 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 407b968b728..bb6e814ad9f 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadNothingStep.cpp b/src/Processors/QueryPlan/ReadNothingStep.cpp index 7019b88f0b2..253f3a5b980 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.cpp +++ b/src/Processors/QueryPlan/ReadNothingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 114fe661c70..acaeb2bc9a7 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 47f8187c3aa..4ca3d0ebf54 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index db14950b229..be2cd2348a4 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index 85705595479..5d40a9e241e 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index ca09f4a9474..cd4bb5f6730 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 7c2b104b61e..3d17c13ad4c 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 2109cb5eba4..23c3ddec401 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 18dbd2e03aa..4974c33f290 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index bf3cafd6ff5..8357a997960 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 8e7a09b320d..839ab0cac88 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index df2ea4b03f0..7f0893a6c0d 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 6956dedbc41..260fdfb3a19 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 57cfa6c01b8..86998614189 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include "IO/CompressionMethod.h" #include "Parsers/ASTLiteral.h" diff --git a/src/QueryPipeline/BlockIO.h b/src/QueryPipeline/BlockIO.h index d699d525f2f..0f05beca4a8 100644 --- a/src/QueryPipeline/BlockIO.h +++ b/src/QueryPipeline/BlockIO.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Processors/Chain.cpp b/src/QueryPipeline/Chain.cpp similarity index 99% rename from src/Processors/Chain.cpp rename to src/QueryPipeline/Chain.cpp index 5e3b2e6a678..ca7b0b80692 100644 --- a/src/Processors/Chain.cpp +++ b/src/QueryPipeline/Chain.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/Chain.h b/src/QueryPipeline/Chain.h similarity index 97% rename from src/Processors/Chain.h rename to src/QueryPipeline/Chain.h index da5167f9c7a..c5fdc34cecf 100644 --- a/src/Processors/Chain.h +++ b/src/QueryPipeline/Chain.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Pipe.cpp b/src/QueryPipeline/Pipe.cpp similarity index 99% rename from src/Processors/Pipe.cpp rename to src/QueryPipeline/Pipe.cpp index acf46b95346..6cef7cc28bd 100644 --- a/src/Processors/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Processors/Pipe.h b/src/QueryPipeline/Pipe.h similarity index 98% rename from src/Processors/Pipe.h rename to src/QueryPipeline/Pipe.h index 3341734430c..0af02a5e662 100644 --- a/src/Processors/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include #include #include diff --git a/src/Processors/PipelineResourcesHolder.cpp b/src/QueryPipeline/PipelineResourcesHolder.cpp similarity index 94% rename from src/Processors/PipelineResourcesHolder.cpp rename to src/QueryPipeline/PipelineResourcesHolder.cpp index 9cb2ea301ad..a4b85ed662b 100644 --- a/src/Processors/PipelineResourcesHolder.cpp +++ b/src/QueryPipeline/PipelineResourcesHolder.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/Processors/PipelineResourcesHolder.h b/src/QueryPipeline/PipelineResourcesHolder.h similarity index 100% rename from src/Processors/PipelineResourcesHolder.h rename to src/QueryPipeline/PipelineResourcesHolder.h diff --git a/src/Processors/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp similarity index 99% rename from src/Processors/QueryPipeline.cpp rename to src/QueryPipeline/QueryPipeline.cpp index 98ac81f7217..ce1c9473f60 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,10 +1,10 @@ #include -#include +#include #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Processors/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h similarity index 98% rename from src/Processors/QueryPipeline.h rename to src/QueryPipeline/QueryPipeline.h index 42fdb429a14..beb46361f95 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp similarity index 99% rename from src/Processors/QueryPipelineBuilder.cpp rename to src/QueryPipeline/QueryPipelineBuilder.cpp index 08a568b48f1..40c64046560 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h similarity index 99% rename from src/Processors/QueryPipelineBuilder.h rename to src/QueryPipeline/QueryPipelineBuilder.h index 7e0ddbc0285..d2bbea03ce5 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index a77a791278a..b01ed7ba9a2 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -7,7 +7,7 @@ #include #include #include "Core/Protocol.h" -#include +#include #include #include #include diff --git a/src/QueryPipeline/narrowBlockInputStreams.cpp b/src/QueryPipeline/narrowBlockInputStreams.cpp index 17e44e38293..19bebe4a0bf 100644 --- a/src/QueryPipeline/narrowBlockInputStreams.cpp +++ b/src/QueryPipeline/narrowBlockInputStreams.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include "narrowBlockInputStreams.h" diff --git a/src/Processors/printPipeline.cpp b/src/QueryPipeline/printPipeline.cpp similarity index 99% rename from src/Processors/printPipeline.cpp rename to src/QueryPipeline/printPipeline.cpp index cbf8cb3a77d..40c88502ed0 100644 --- a/src/Processors/printPipeline.cpp +++ b/src/QueryPipeline/printPipeline.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Processors/printPipeline.h b/src/QueryPipeline/printPipeline.h similarity index 100% rename from src/Processors/printPipeline.h rename to src/QueryPipeline/printPipeline.h diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index 7625fe8fa09..d39c49bb61c 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -4,10 +4,10 @@ #include #include #include -#include +#include #include #include -#include +#include using namespace DB; diff --git a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp index 04ccc64fd7c..751f7ef8635 100644 --- a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp +++ b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 25085f38889..ba2644e0fba 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 3c0b6333fc1..668cec22e8b 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 12558054d81..6e81f5577ab 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -24,8 +24,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8effb5ed9bf..7c158794caf 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ec6e58f3fbb..74e17442fe8 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 6cf7ce59fa2..f010ef50ecd 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 23ea280e88c..729b545e9a0 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 917ea156ab7..fa36588513b 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 4e9e2c450b1..f771b2239ef 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 42294b8152c..19f1d7780b0 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.h b/src/Storages/ReadFinalForExternalReplicaStorage.h index f09a115919d..1be0aa0f4a6 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.h +++ b/src/Storages/ReadFinalForExternalReplicaStorage.h @@ -7,7 +7,7 @@ #if USE_MYSQL || USE_LIBPQXX #include -#include +#include namespace DB diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 3a37e57e008..18bf0e2c19b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -17,7 +17,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index e305d4c6183..349d894b2f1 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 21784952c23..76be3353808 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index cedb7d26b48..0af8fe6f7df 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4ae55272db6..7c6543c5519 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -36,7 +36,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cfec328bbfc..a4dfbfc3f96 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 1138794adb0..2ed7a77b59d 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index bfb634c6bba..b28bc143bb0 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index c2c1cff5f53..8ac341661bf 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -17,7 +17,7 @@ #include #include -#include +#include #include #include /// toLower diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ecf3da83ea6..41d178fc04d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -23,7 +23,7 @@ #include "StorageLogSettings.h" #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index 52f53b9ceee..37a9838593b 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 299e39a3836..37cb238ba0f 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ab42da1dfa0..acc40787fed 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,7 +27,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 15430f60285..146fd8b616a 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index a1724af38cb..50d2124958c 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 96e72c77f00..de667c1d75c 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index c81ef6febdc..62e0ccc76aa 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 65e07fa6144..bc03dbb45ae 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c94c519c3b9..8cbec46d0e0 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,7 +35,7 @@ #include #include -#include +#include #include #include @@ -52,7 +52,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index b78a09ebcc4..9d05235552c 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include "Processors/Sources/SourceWithProgress.h" #include #include diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index cc5eb835438..50113c391cc 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -31,7 +31,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 557f378ab77..0b7ab30fa24 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index d1778342ec5..74d922d5594 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -36,7 +36,7 @@ #include #include -#include +#include #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9397986fadd..174ee58ee42 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -19,7 +19,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index ace5ca3667c..650782afbba 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index df774554365..9ffea587b97 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -14,7 +14,7 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a0924896437..a90e21a2edb 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index c074659af2b..6897b4a3a79 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 3a88cc96639..136c2489be2 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 7c28f897121..7558ae0ae92 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 254e6f77e0c..f32a609077f 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index d1456d72685..624fc54998c 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 5b891c43aae..f86295cd06b 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -18,8 +18,8 @@ #include #include #include -#include -#include +#include +#include #if !defined(__clang__) # pragma GCC diagnostic push From fa92fd28bf23dd9a0522020d44ee1cb67e08679b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 16 Oct 2021 17:34:31 +0300 Subject: [PATCH 77/89] Update LocalServer.cpp --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 10c4bf592ca..766123339af 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -129,7 +129,7 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) { if (current_exception) current_exception->rethrow(); - continue; + return true; } case MultiQueryProcessingStage::EXECUTE_QUERY: { From 3102d24872e1579dd28d2133171cb6e2d4aceccc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Oct 2021 10:38:11 +0300 Subject: [PATCH 78/89] Fix --hung-check in clickhouse-test JSONEachRow cannot be parsed with a simple json.loads(), instead it should be passed to json.loads() line by line. Fixes: #30065 --- tests/clickhouse-test | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b2a9358371a..7b934844dc4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -109,7 +109,10 @@ def clickhouse_execute_json(base_args, query, timeout=30, settings=None): data = clickhouse_execute_http(base_args, query, timeout, settings, 'JSONEachRow') if not data: return None - return json.loads(data) + rows = [] + for row in data.strip().split(b'\n'): + rows.append(json.loads(row)) + return rows class Terminated(KeyboardInterrupt): From c6e61e9497f708589eaf0602a9727943c015190b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 17:58:19 +0300 Subject: [PATCH 79/89] clickhouse-test: use splitlines() over split('\n') --- tests/clickhouse-test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7b934844dc4..6512c40c44b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -110,7 +110,7 @@ def clickhouse_execute_json(base_args, query, timeout=30, settings=None): if not data: return None rows = [] - for row in data.strip().split(b'\n'): + for row in data.strip().splitlines(): rows.append(json.loads(row)) return rows @@ -478,19 +478,19 @@ class TestCase: if os.path.isfile(self.stdout_file): description += ", result:\n\n" - description += '\n'.join(open(self.stdout_file).read().split('\n')[:100]) + description += '\n'.join(open(self.stdout_file).read().splitlines()[:100]) description += '\n' description += "\nstdout:\n{}\n".format(stdout) return TestResult(self.name, TestStatus.FAIL, reason, total_time, description) if stderr: - description += "\n{}\n".format('\n'.join(stderr.split('\n')[:100])) + description += "\n{}\n".format('\n'.join(stderr.splitlines()[:100])) description += "\nstdout:\n{}\n".format(stdout) return TestResult(self.name, TestStatus.FAIL, FailureReason.STDERR, total_time, description) if 'Exception' in stdout: - description += "\n{}\n".format('\n'.join(stdout.split('\n')[:100])) + description += "\n{}\n".format('\n'.join(stdout.splitlines()[:100])) return TestResult(self.name, TestStatus.FAIL, FailureReason.EXCEPTION, total_time, description) if '@@SKIP@@' in stdout: From 612a21cc9775590900c40ebe987da6d9571b2cf2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 17:59:17 +0300 Subject: [PATCH 80/89] clickhouse-test: do not propagate CLICKHOUSE_PORT_HTTP to clickhouse-client --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6512c40c44b..1e8c2bfb8ad 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1395,7 +1395,6 @@ if __name__ == '__main__': http_port = os.getenv("CLICKHOUSE_PORT_HTTP") if http_port is not None: args.http_port = int(http_port) - args.client += f" --port={http_port}" else: args.http_port = 8123 From 434de66a2b6f34eb8f8fc4704eded45bb0763fd0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 12:59:01 -0300 Subject: [PATCH 81/89] Doc. ArgMax/Min are not supported anymore by SimpleAggregateFunction --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 7b81c577762..8f47bd0902b 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -21,8 +21,6 @@ - [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) - [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap) - [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) -- [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md) -- [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md) !!! note "Примечание" Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому комбинаторы [-Merge](../../sql-reference/aggregate-functions/combinators.md#aggregate_functions_combinators-merge) и [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) не требуются. From 329d81a0efb7fab70fe91a8da5cd0ee36c47ad82 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 16 Oct 2021 20:03:45 +0300 Subject: [PATCH 82/89] Various fixes to install procedure --- programs/install/Install.cpp | 516 +++++++++++++++++++++-------------- tests/CMakeLists.txt | 22 +- 2 files changed, 323 insertions(+), 215 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index e001d0ceb53..6b2d3a58a22 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -66,6 +66,7 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int SYSTEM_ERROR; extern const int NOT_ENOUGH_SPACE; + extern const int NOT_IMPLEMENTED; extern const int CANNOT_KILL; } @@ -75,8 +76,17 @@ namespace ErrorCodes #define HILITE "\033[1m" #define END_HILITE "\033[0m" -static constexpr auto CLICKHOUSE_BRIDGE_USER = "clickhouse-bridge"; -static constexpr auto CLICKHOUSE_BRIDGE_GROUP = "clickhouse-bridge"; +#if defined(OS_DARWIN) +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = ""; +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = ""; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = ""; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_GROUP = ""; +#else +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = "clickhouse"; +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = "clickhouse"; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = "clickhouse-bridge"; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_GROUP = "clickhouse-bridge"; +#endif using namespace DB; namespace po = boost::program_options; @@ -127,36 +137,77 @@ static bool filesEqual(std::string path1, std::string path2) && 0 == memcmp(in1.buffer().begin(), in2.buffer().begin(), in1.buffer().size()); } +static void changeOwnership(const String & file_name, const String & user_name, const String & group_name = {}, bool recursive = true) +{ + if (!user_name.empty() || !group_name.empty()) + { + std::string command = fmt::format("chown {} {}:{} '{}'", (recursive ? "-R" : ""), user_name, group_name, file_name); + fmt::print(" {}\n", command); + executeScript(command); + } +} + +static void createGroup(const String & group_name) +{ + if (!group_name.empty()) + { +#if defined(OS_DARWIN) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a group in macOS"); +#else + std::string command = fmt::format("groupadd -r {}", group_name); + fmt::print(" {}\n", command); + executeScript(command); +#endif + } +} + +static void createUser(const String & user_name, [[maybe_unused]] const String & group_name) +{ + if (!user_name.empty()) + { +#if defined(OS_DARWIN) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a user in macOS"); +#else + std::string command = group_name.empty() + ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user_name) + : fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent -g {} {}", group_name, user_name); + fmt::print(" {}\n", command); + executeScript(command); +#endif + } +} + int mainEntryClickHouseInstall(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("prefix", po::value()->default_value(""), "prefix for all paths") - ("binary-path", po::value()->default_value("/usr/bin"), "where to install binaries") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "where to install configs") - ("log-path", po::value()->default_value("/var/log/clickhouse-server"), "where to create log directory") - ("data-path", po::value()->default_value("/var/lib/clickhouse"), "directory for data") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user to create") - ("group", po::value()->default_value("clickhouse"), "clickhouse group to create") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " install [options]\n"; - std::cout << desc << '\n'; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "where to install binaries") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "where to install configs") + ("log-path", po::value()->default_value("var/log/clickhouse-server"), "where to create log directory") + ("data-path", po::value()->default_value("var/lib/clickhouse"), "directory for data") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user to create") + ("group", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group to create") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " install [options]\n"; + std::cout << desc << '\n'; + return 1; + } + /// We need to copy binary to the binary directory. /// The binary is currently run. We need to obtain its path from procfs (on Linux). @@ -171,6 +222,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (res != 0) Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); + if (path.back() == '\0') + path.pop_back(); + fs::path binary_self_path(path); #else fs::path binary_self_path = "/proc/self/exe"; @@ -186,8 +240,8 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// TODO An option to link instead of copy - useful for developers. - fs::path prefix = fs::path(options["prefix"].as()); - fs::path bin_dir = prefix / fs::path(options["binary-path"].as()); + fs::path prefix = options["prefix"].as(); + fs::path bin_dir = prefix / options["binary-path"].as(); fs::path main_bin_path = bin_dir / "clickhouse"; fs::path main_bin_tmp_path = bin_dir / "clickhouse.new"; @@ -225,6 +279,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { + if (!fs::exists(bin_dir)) + { + fmt::print("Creating binary directory {}.\n", bin_dir.string()); + fs::create_directories(bin_dir); + } + size_t available_space = fs::space(bin_dir).available; if (available_space < binary_size) throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space for clickhouse binary in {}, required {}, available {}.", @@ -326,34 +386,18 @@ int mainEntryClickHouseInstall(int argc, char ** argv) std::string user = options["user"].as(); std::string group = options["group"].as(); - auto create_group = [](const String & group_name) - { - std::string command = fmt::format("groupadd -r {}", group_name); - fmt::print(" {}\n", command); - executeScript(command); - }; - if (!group.empty()) { fmt::print("Creating clickhouse group if it does not exist.\n"); - create_group(group); + createGroup(group); } else - fmt::print("Will not create clickhouse group"); - - auto create_user = [](const String & user_name, const String & group_name) - { - std::string command = group_name.empty() - ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user_name) - : fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent -g {} {}", group_name, user_name); - fmt::print(" {}\n", command); - executeScript(command); - }; + fmt::print("Will not create a dedicated clickhouse group.\n"); if (!user.empty()) { fmt::print("Creating clickhouse user if it does not exist.\n"); - create_user(user, group); + createUser(user, group); if (group.empty()) group = user; @@ -361,6 +405,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Setting ulimits. try { +#if defined(OS_DARWIN) + + /// TODO Set ulimits on macOS. + +#else fs::path ulimits_dir = "/etc/security/limits.d"; fs::path ulimits_file = ulimits_dir / fmt::format("{}.conf", user); fmt::print("Will set ulimits for {} user in {}.\n", user, ulimits_file.string()); @@ -374,16 +423,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out.write(ulimits_content.data(), ulimits_content.size()); out.sync(); out.finalize(); +#endif } catch (...) { std::cerr << "Cannot set ulimits: " << getCurrentExceptionMessage(false) << "\n"; } - - /// TODO Set ulimits on Mac OS X } else - fmt::print("Will not create clickhouse user.\n"); + fmt::print("Will not create a dedicated clickhouse user.\n"); /// Creating configuration files and directories. @@ -400,9 +448,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fs::path config_d = config_dir / "config.d"; fs::path users_d = config_dir / "users.d"; - std::string log_path = prefix / options["log-path"].as(); - std::string data_path = prefix / options["data-path"].as(); - std::string pid_path = prefix / options["pid-path"].as(); + fs::path log_path = prefix / options["log-path"].as(); + fs::path data_path = prefix / options["data-path"].as(); + fs::path pid_path = prefix / options["pid-path"].as(); bool has_password_for_default_user = false; @@ -427,10 +475,78 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - WriteBufferFromFile out(main_config_file.string()); - out.write(main_config_content.data(), main_config_content.size()); - out.sync(); - out.finalize(); + { + WriteBufferFromFile out(main_config_file.string()); + out.write(main_config_content.data(), main_config_content.size()); + out.sync(); + out.finalize(); + } + + /// Override the default paths. + + /// Data paths. + { + std::string data_file = config_d / "data-paths.xml"; + WriteBufferFromFile out(data_file); + out << "\n" + " " << data_path.string() << "\n" + " " << (data_path / "tmp").string() << "\n" + " " << (data_path / "user_files").string() << "\n" + " " << (data_path / "format_schemas").string() << "\n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Data path configuration override is saved to file {}.\n", data_file); + } + + /// Logger. + { + std::string logger_file = config_d / "logger.xml"; + WriteBufferFromFile out(logger_file); + out << "\n" + " \n" + " " << (log_path / "clickhouse-server.log").string() << "\n" + " " << (log_path / "clickhouse-server.err.log").string() << "\n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Log path configuration override is saved to file {}.\n", logger_file); + } + + /// User directories. + { + std::string user_directories_file = config_d / "user-directories.xml"; + WriteBufferFromFile out(user_directories_file); + out << "\n" + " \n" + " \n" + " " << (data_path / "access").string() << "\n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("User directory path configuration override is saved to file {}.\n", user_directories_file); + } + + /// OpenSSL. + { + std::string openssl_file = config_d / "openssl.xml"; + WriteBufferFromFile out(openssl_file); + out << "\n" + " \n" + " \n" + " " << (config_dir / "server.crt").string() << "\n" + " " << (config_dir / "server.key").string() << "\n" + " " << (config_dir / "dhparam.pem").string() << "\n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("OpenSSL path configuration override is saved to file {}.\n", openssl_file); + } } } else @@ -443,13 +559,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (configuration->has("path")) { data_path = configuration->getString("path"); - fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); + fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path.string()); } if (configuration->has("logger.log")) { log_path = fs::path(configuration->getString("logger.log")).remove_filename(); - fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); + fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path.string()); } } @@ -485,82 +601,44 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } - auto change_ownership = [](const String & file_name, const String & user_name, const String & group_name) - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user_name, group_name, file_name); - fmt::print(" {}\n", command); - executeScript(command); - }; - - /// Chmod and chown configs - change_ownership(config_dir.string(), user, group); - - /// Symlink "preprocessed_configs" is created by the server, so "write" is needed. - fs::permissions(config_dir, fs::perms::owner_all, fs::perm_options::replace); - - /// Subdirectories, so "execute" is needed. - if (fs::exists(config_d)) - fs::permissions(config_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); - if (fs::exists(users_d)) - fs::permissions(users_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); - - /// Readonly. - if (fs::exists(main_config_file)) - fs::permissions(main_config_file, fs::perms::owner_read, fs::perm_options::replace); - if (fs::exists(users_config_file)) - fs::permissions(users_config_file, fs::perms::owner_read, fs::perm_options::replace); - /// Create directories for data and log. if (fs::exists(log_path)) { - fmt::print("Log directory {} already exists.\n", log_path); + fmt::print("Log directory {} already exists.\n", log_path.string()); } else { - fmt::print("Creating log directory {}.\n", log_path); + fmt::print("Creating log directory {}.\n", log_path.string()); fs::create_directories(log_path); } if (fs::exists(data_path)) { - fmt::print("Data directory {} already exists.\n", data_path); + fmt::print("Data directory {} already exists.\n", data_path.string()); } else { - fmt::print("Creating data directory {}.\n", data_path); + fmt::print("Creating data directory {}.\n", data_path.string()); fs::create_directories(data_path); } if (fs::exists(pid_path)) { - fmt::print("Pid directory {} already exists.\n", pid_path); + fmt::print("Pid directory {} already exists.\n", pid_path.string()); } else { - fmt::print("Creating pid directory {}.\n", pid_path); + fmt::print("Creating pid directory {}.\n", pid_path.string()); fs::create_directories(pid_path); } /// Chmod and chown data and log directories - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, log_path); - fmt::print(" {}\n", command); - executeScript(command); - } + changeOwnership(log_path, user, group); + changeOwnership(pid_path, user, group); - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, pid_path); - fmt::print(" {}\n", command); - executeScript(command); - } - - { - /// Not recursive, because there can be a huge number of files and it will be slow. - std::string command = fmt::format("chown {}:{} '{}'", user, group, data_path); - fmt::print(" {}\n", command); - executeScript(command); - } + /// Not recursive, because there can be a huge number of files and it will be slow. + changeOwnership(data_path, user, group, /* recursive= */ false); /// All users are allowed to read pid file (for clickhouse status command). fs::permissions(pid_path, fs::perms::owner_all | fs::perms::group_read | fs::perms::others_read, fs::perm_options::replace); @@ -576,13 +654,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (fs::exists(odbc_bridge_path) || fs::exists(library_bridge_path)) { - create_group(CLICKHOUSE_BRIDGE_GROUP); - create_user(CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + createGroup(DEFAULT_CLICKHOUSE_BRIDGE_GROUP); + createUser(DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); if (fs::exists(odbc_bridge_path)) - change_ownership(odbc_bridge_path, CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + changeOwnership(odbc_bridge_path, DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); if (fs::exists(library_bridge_path)) - change_ownership(library_bridge_path, CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + changeOwnership(library_bridge_path, DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); } bool stdin_is_a_tty = isatty(STDIN_FILENO); @@ -701,6 +779,25 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } + /// Chmod and chown configs + changeOwnership(config_dir, user, group); + + /// Symlink "preprocessed_configs" is created by the server, so "write" is needed. + fs::permissions(config_dir, fs::perms::owner_all, fs::perm_options::replace); + + /// Subdirectories, so "execute" is needed. + if (fs::exists(config_d)) + fs::permissions(config_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + if (fs::exists(users_d)) + fs::permissions(users_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + + /// Readonly. + if (fs::exists(main_config_file)) + fs::permissions(main_config_file, fs::perms::owner_read, fs::perm_options::replace); + if (fs::exists(users_config_file)) + fs::permissions(users_config_file, fs::perms::owner_read, fs::perm_options::replace); + + std::string maybe_password; if (has_password_for_default_user) maybe_password = " --password"; @@ -766,11 +863,7 @@ namespace /// All users are allowed to read pid file (for clickhouse status command). fs::permissions(pid_path, fs::perms::owner_all | fs::perms::group_read | fs::perms::others_read, fs::perm_options::replace); - { - std::string command = fmt::format("chown --recursive {} '{}'", user, pid_path.string()); - fmt::print(" {}\n", command); - executeScript(command); - } + changeOwnership(pid_path, user); } std::string command = fmt::format("{} --config-file {} --pid-file {} --daemon", @@ -974,34 +1067,36 @@ namespace int mainEntryClickHouseStart(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " start\n"; - return 1; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " start\n"; + return 1; + } + std::string user = options["user"].as(); - fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; - fs::path config = fs::path(options["config-path"].as()) / "config.xml"; - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + fs::path prefix = options["prefix"].as(); + fs::path executable = prefix / options["binary-path"].as() / "clickhouse-server"; + fs::path config = prefix / options["config-path"].as() / "config.xml"; + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; return start(user, executable, config, pid_file); } @@ -1015,28 +1110,30 @@ int mainEntryClickHouseStart(int argc, char ** argv) int mainEntryClickHouseStop(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("force", po::bool_switch(), "Stop with KILL signal instead of TERM") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " stop\n"; - return 1; - } - try { - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("force", po::bool_switch(), "Stop with KILL signal instead of TERM") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " stop\n"; + return 1; + } + + fs::path prefix = options["prefix"].as(); + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; return stop(pid_file, options["force"].as()); } @@ -1050,72 +1147,79 @@ int mainEntryClickHouseStop(int argc, char ** argv) int mainEntryClickHouseStatus(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " status\n"; - return 1; - } - try { - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " status\n"; + return 1; + } + + fs::path prefix = options["prefix"].as(); + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; + isRunning(pid_file); - return 0; } catch (...) { std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } + + return 0; } int mainEntryClickHouseRestart(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user") - ("force", po::value()->default_value(false), "Stop with KILL signal instead of TERM") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " restart\n"; - return 1; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") + ("force", po::value()->default_value(false), "Stop with KILL signal instead of TERM") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " restart\n"; + return 1; + } + std::string user = options["user"].as(); - fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; - fs::path config = fs::path(options["config-path"].as()) / "config.xml"; - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + fs::path prefix = options["prefix"].as(); + fs::path executable = prefix / options["binary-path"].as() / "clickhouse-server"; + fs::path config = prefix / options["config-path"].as() / "config.xml"; + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; if (int res = stop(pid_file, options["force"].as())) return res; + return start(user, executable, config, pid_file); } catch (...) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 132909438da..c9858910837 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -7,15 +7,19 @@ else () include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) endif () -install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) -install ( - DIRECTORY queries performance config - DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test - USE_SOURCE_PERMISSIONS - COMPONENT clickhouse - PATTERN "CMakeLists.txt" EXCLUDE - PATTERN ".gitignore" EXCLUDE -) +option (ENABLE_CLICKHOUSE_TEST "Install clickhouse-test script and relevant tests scenarios" ON) + +if (ENABLE_CLICKHOUSE_TEST) + install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + install ( + DIRECTORY queries performance config + DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test + USE_SOURCE_PERMISSIONS + COMPONENT clickhouse + PATTERN "CMakeLists.txt" EXCLUDE + PATTERN ".gitignore" EXCLUDE + ) +endif () if (ENABLE_TEST_INTEGRATION) add_subdirectory (integration) From 8b0359584afa84a310f43c37aeb01caf1ec7450a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:08:37 +0000 Subject: [PATCH 83/89] Update test --- tests/queries/0_stateless/01196_max_parser_depth.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01196_max_parser_depth.reference b/tests/queries/0_stateless/01196_max_parser_depth.reference index a72c1b18aa2..072fc270acd 100644 --- a/tests/queries/0_stateless/01196_max_parser_depth.reference +++ b/tests/queries/0_stateless/01196_max_parser_depth.reference @@ -1,3 +1,4 @@ Code: 306 Code: 306 Code: 306 +Code: 306 From 443efe8f5c207689f0575d3c79b03d9589b2d191 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 16:21:18 -0300 Subject: [PATCH 84/89] Update quotas.md --- docs/ru/operations/system-tables/quotas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/system-tables/quotas.md b/docs/ru/operations/system-tables/quotas.md index 3715bc89596..6c8b5a3eebf 100644 --- a/docs/ru/operations/system-tables/quotas.md +++ b/docs/ru/operations/system-tables/quotas.md @@ -11,7 +11,7 @@ - `[]` — Все пользователи используют одну и ту же квоту. - `['user_name']` — Соединения с одинаковым именем пользователя используют одну и ту же квоту. - `['ip_address']` — Соединения с одинаковым IP-адресом используют одну и ту же квоту. - - `['client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Ключ может быть явно задан клиентом. При использовании [clickhouse-client](../../interfaces/cli.md), передайте ключевое значение в параметре `--quota-key`, или используйте параметр `quota_key` файле настроек клиента. В случае использования HTTP интерфейса, используйте заголовок `X-ClickHouse-Quota`. + - `['client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Ключ может быть явно задан клиентом. При использовании [clickhouse-client](../../interfaces/cli.md), передайте ключевое значение в параметре `--quota_key`, или используйте параметр `quota_key` файле настроек клиента. В случае использования HTTP интерфейса, используйте заголовок `X-ClickHouse-Quota`. - `['user_name', 'client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `user_name`. - `['client_key', 'ip_address']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `ip_address`. - `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Длины временных интервалов для расчета потребления ресурсов, в секундах. From 5e0e0a3946390736f631a94ebf5b88a0588662d6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 16:22:19 -0300 Subject: [PATCH 85/89] Update quotas.md --- docs/en/operations/system-tables/quotas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/quotas.md b/docs/en/operations/system-tables/quotas.md index d7a73de5046..bdcc13340f0 100644 --- a/docs/en/operations/system-tables/quotas.md +++ b/docs/en/operations/system-tables/quotas.md @@ -10,7 +10,7 @@ Columns: - `[]` — All users share the same quota. - `['user_name']` — Connections with the same user name share the same quota. - `['ip_address']` — Connections from the same IP share the same quota. - - `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota-key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header. + - `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota_key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header. - `['user_name', 'client_key']` — Connections with the same `client_key` share the same quota. If a key isn’t provided by a client, the qouta is tracked for `user_name`. - `['client_key', 'ip_address']` — Connections with the same `client_key` share the same quota. If a key isn’t provided by a client, the qouta is tracked for `ip_address`. - `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Time interval lengths in seconds. From 4390dde76cfea689602bc4c599dda626cdea3e52 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:48:51 +0000 Subject: [PATCH 86/89] Fix local break on timeout --- src/Client/ClientBase.cpp | 3 ++- src/Client/Connection.h | 2 ++ src/Client/IServerConnection.h | 8 ++++++++ src/Client/LocalConnection.h | 2 ++ 4 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e030d8994d5..0af952d666e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -517,6 +517,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) const size_t poll_interval = std::max(min_poll_interval, std::min(receive_timeout.totalMicroseconds(), default_poll_interval)); + bool break_on_timeout = connection->getConnectionType() != IServerConnection::Type::LOCAL; while (true) { Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE); @@ -547,7 +548,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) else { double elapsed = receive_watch.elapsedSeconds(); - if (elapsed > receive_timeout.totalSeconds()) + if (break_on_timeout && elapsed > receive_timeout.totalSeconds()) { std::cout << "Timeout exceeded while receiving data from server." << " Waited for " << static_cast(elapsed) << " seconds," diff --git a/src/Client/Connection.h b/src/Client/Connection.h index b6054941aeb..848b848a104 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -60,6 +60,8 @@ public: ~Connection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::SERVER; } + static ServerConnectionPtr createConnection(const ConnectionParameters & parameters, ContextPtr context); /// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic. diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 42886c72182..5a853126982 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -56,6 +56,14 @@ class IServerConnection : boost::noncopyable public: virtual ~IServerConnection() = default; + enum class Type + { + SERVER, + LOCAL + }; + + virtual Type getConnectionType() const = 0; + virtual void setDefaultDatabase(const String & database) = 0; virtual void getServerVersion( diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 242d23ddc36..2bd812f200f 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -56,6 +56,8 @@ public: ~LocalConnection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::LOCAL; } + static ServerConnectionPtr createConnection(const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false); void setDefaultDatabase(const String & database) override; From 5fc2279adf5cc901b37ccc57155857ba70d3ce9d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 01:23:55 +0300 Subject: [PATCH 87/89] Add RISC-V build --- cmake/linux/toolchain-riscv64.cmake | 32 +++++++++++++++++++++++++++++ contrib/sysroot | 2 +- 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 cmake/linux/toolchain-riscv64.cmake diff --git a/cmake/linux/toolchain-riscv64.cmake b/cmake/linux/toolchain-riscv64.cmake new file mode 100644 index 00000000000..1ccbd3ee0da --- /dev/null +++ b/cmake/linux/toolchain-riscv64.cmake @@ -0,0 +1,32 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "riscv64") +set (CMAKE_C_COMPILER_TARGET "riscv64-linux-gnu") +set (CMAKE_CXX_COMPILER_TARGET "riscv64-linux-gnu") +set (CMAKE_ASM_COMPILER_TARGET "riscv64-linux-gnu") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-riscv64") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") + +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/contrib/sysroot b/contrib/sysroot index e4663925b73..6172893931e 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit e4663925b73beb57dd29154844c8d50441146753 +Subproject commit 6172893931e19b028f9cabb7095a44361be863df From ceb98cf42ff20032fadc23698ea031aec7ab4483 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 17 Oct 2021 03:04:53 +0400 Subject: [PATCH 88/89] Add comments --- programs/install/Install.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 6b2d3a58a22..606af7ecd0d 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -77,6 +77,7 @@ namespace ErrorCodes #define END_HILITE "\033[0m" #if defined(OS_DARWIN) +/// Until createUser() and createGroup() are implemented, only sudo-less installations are supported/default for macOS. static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = ""; static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = ""; static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = ""; @@ -152,6 +153,9 @@ static void createGroup(const String & group_name) if (!group_name.empty()) { #if defined(OS_DARWIN) + + // TODO: implement. + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a group in macOS"); #else std::string command = fmt::format("groupadd -r {}", group_name); @@ -166,6 +170,9 @@ static void createUser(const String & user_name, [[maybe_unused]] const String & if (!user_name.empty()) { #if defined(OS_DARWIN) + + // TODO: implement. + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a user in macOS"); #else std::string command = group_name.empty() From 0285a15aeca720c4ecccb16755b9d0bd81de0d45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 05:52:00 +0300 Subject: [PATCH 89/89] Minor changes to install script --- docs/_includes/install/universal.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index 7cba682e772..eaea4cc69ed 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -47,13 +47,17 @@ then fi URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" +echo echo "Will download ${URL}" +echo curl -O "${URL}" && chmod a+x clickhouse && +echo echo "Successfully downloaded the ClickHouse binary, you can run it as: ./clickhouse" if [ "${OS}" = "Linux" ] then + echo echo "You can also install it: sudo ./clickhouse install" fi