From 3410082cb22b6803a563b65e24e1f66523282a2e Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Wed, 29 Mar 2023 19:58:25 +0800 Subject: [PATCH 01/52] add columns perform_ttl_move_on_insert, load_balancing for table system.storage_policies --- .../system-tables/storage_policies.md | 8 +++++ .../System/StorageSystemStoragePolicies.cpp | 29 ++++++++++++++++--- .../02117_show_create_table_system.reference | 6 ++-- 3 files changed, 37 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/system-tables/storage_policies.md b/docs/en/operations/system-tables/storage_policies.md index 966b677c7e3..69e0f7f0a55 100644 --- a/docs/en/operations/system-tables/storage_policies.md +++ b/docs/en/operations/system-tables/storage_policies.md @@ -11,8 +11,16 @@ Columns: - `volume_name` ([String](../../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. - `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration, the data fills the volumes according this priority, i.e. data during inserts and merges is written to volumes with a lower priority (taking into account other rules: TTL, `max_data_part_size`, `move_factor`). - `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. +- `volume_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of volume. Can have one of the following values: + - `JBOD` + - `SINGLE_DISK` + - `UNKNOWN` - `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). - `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. - `prefer_not_to_merge` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Value of the `prefer_not_to_merge` setting. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks. +- `perform_ttl_move_on_insert` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Value of the `perform_ttl_move_on_insert` setting. — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3). +- `load_balancing` ([Enum8](../../sql-reference/data-types/enum.md)) — Policy for disk balancing. Can have one of the following values: + - `ROUND_ROBIN` + - `LEAST_USED` If the storage policy contains more then one volume, then information for each volume is stored in the individual row of the table. diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index b42bd7859dd..39e99884e1d 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -17,6 +18,18 @@ namespace ErrorCodes { } +namespace +{ + template + DataTypeEnum8::Values getTypeEnumValues() + { + DataTypeEnum8::Values enum_values; + for (auto value : magic_enum::enum_values()) + enum_values.emplace_back(magic_enum::enum_name(value), magic_enum::enum_integer(value)); + return enum_values; + } +} + StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & table_id_) : IStorage(table_id_) @@ -28,10 +41,12 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & tab {"volume_name", std::make_shared()}, {"volume_priority", std::make_shared()}, {"disks", std::make_shared(std::make_shared())}, - {"volume_type", std::make_shared()}, + {"volume_type", std::make_shared(getTypeEnumValues())}, {"max_data_part_size", std::make_shared()}, {"move_factor", std::make_shared()}, - {"prefer_not_to_merge", std::make_shared()} + {"prefer_not_to_merge", std::make_shared()}, + {"perform_ttl_move_on_insert", std::make_shared()}, + {"load_balancing", std::make_shared(getTypeEnumValues())} })); // TODO: Add string column with custom volume-type-specific options setInMemoryMetadata(storage_metadata); @@ -52,10 +67,12 @@ Pipe StorageSystemStoragePolicies::read( MutableColumnPtr col_volume_name = ColumnString::create(); MutableColumnPtr col_priority = ColumnUInt64::create(); MutableColumnPtr col_disks = ColumnArray::create(ColumnString::create()); - MutableColumnPtr col_volume_type = ColumnString::create(); + MutableColumnPtr col_volume_type = ColumnInt8::create(); MutableColumnPtr col_max_part_size = ColumnUInt64::create(); MutableColumnPtr col_move_factor = ColumnFloat32::create(); MutableColumnPtr col_prefer_not_to_merge = ColumnUInt8::create(); + MutableColumnPtr col_perform_ttl_move_on_insert = ColumnUInt8::create(); + MutableColumnPtr col_load_balancing = ColumnInt8::create(); for (const auto & [policy_name, policy_ptr] : context->getPoliciesMap()) { @@ -70,10 +87,12 @@ Pipe StorageSystemStoragePolicies::read( for (const auto & disk_ptr : volumes[i]->getDisks()) disks.push_back(disk_ptr->getName()); col_disks->insert(disks); - col_volume_type->insert(magic_enum::enum_name(volumes[i]->getType())); + col_volume_type->insert(static_cast(volumes[i]->getType())); col_max_part_size->insert(volumes[i]->max_data_part_size); col_move_factor->insert(policy_ptr->getMoveFactor()); col_prefer_not_to_merge->insert(volumes[i]->areMergesAvoided() ? 1 : 0); + col_perform_ttl_move_on_insert->insert(volumes[i]->perform_ttl_move_on_insert); + col_load_balancing->insert(static_cast(volumes[i]->load_balancing)); } } @@ -86,6 +105,8 @@ Pipe StorageSystemStoragePolicies::read( res_columns.emplace_back(std::move(col_max_part_size)); res_columns.emplace_back(std::move(col_move_factor)); res_columns.emplace_back(std::move(col_prefer_not_to_merge)); + res_columns.emplace_back(std::move(col_perform_ttl_move_on_insert)); + res_columns.emplace_back(std::move(col_load_balancing)); UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 6fb59657305..f67d22dffdd 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1024,10 +1024,12 @@ CREATE TABLE system.storage_policies `volume_name` String, `volume_priority` UInt64, `disks` Array(String), - `volume_type` String, + `volume_type` Enum8('JBOD' = 0, 'SINGLE_DISK' = 1, 'UNKNOWN' = 2), `max_data_part_size` UInt64, `move_factor` Float32, - `prefer_not_to_merge` UInt8 + `prefer_not_to_merge` UInt8, + `perform_ttl_move_on_insert` UInt8, + `load_balancing` Enum8('ROUND_ROBIN' = 0, 'LEAST_USED' = 1) ) ENGINE = SystemStoragePolicies COMMENT 'SYSTEM TABLE is built on the fly.' From b199219ec1b4f34bfa0b98983e3efb1103785b35 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 12:24:22 +0000 Subject: [PATCH 02/52] Add reading step for system zookeeper. Analyze path from filter DAG. --- .../System/StorageSystemZooKeeper.cpp | 247 ++++++++++-------- src/Storages/System/StorageSystemZooKeeper.h | 17 +- 2 files changed, 154 insertions(+), 110 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 9663c76a5c3..ee33253e40f 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -12,11 +12,17 @@ #include #include #include +#include +#include +#include +#include #include #include #include #include #include +#include +#include #include #include #include @@ -155,8 +161,24 @@ public: } }; +class ReadFromSystemZooKeeper final : public SourceStepWithFilter +{ +public: + ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info_, ContextPtr context_); + + String getName() const override { return "ReadFromSystemZooKeeper"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + +private: + void fillData(MutableColumns & res_columns) const; + + std::shared_ptr storage_limits; + ContextPtr context; +}; + StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; ColumnsDescription desc; @@ -173,6 +195,26 @@ StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } +bool StorageSystemZooKeeper::mayBenefitFromIndexForIn(const ASTPtr & node, ContextPtr, const StorageMetadataPtr &) const +{ + return node->as() && node->getColumnName() == "path"; +} + +void StorageSystemZooKeeper::read( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) +{ + auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto read_step = std::make_unique(header, query_info, context); + query_plan.addStep(std::move(read_step)); +} + SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context) { if (!context->getConfigRef().getBool("allow_zookeeper_write", false)) @@ -229,125 +271,99 @@ static String pathCorrected(const String & path) return path_corrected; } - -static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, bool allow_unrestricted) +static bool isPathNode(const ActionsDAG::Node * node) { - const auto * function = elem.as(); - if (!function) - return false; + while (node->type == ActionsDAG::ActionType::ALIAS) + node = node->children.at(0); - if (function->name == "and") + return node->result_name == "path"; +} + +static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextPtr context, bool allow_unrestricted) +{ + if (node.type != ActionsDAG::ActionType::FUNCTION) + return; + + auto function_name = node.function_base->getName(); + if (function_name == "and") { - for (const auto & child : function->arguments->children) - if (extractPathImpl(*child, res, context, allow_unrestricted)) - return true; + for (const auto * child : node.children) + extractPathImpl(*child, res, context, allow_unrestricted); - return false; + return; } - const auto & args = function->arguments->as(); - if (args.children.size() != 2) - return false; + if (node.children.size() != 2) + return; - if (function->name == "in") + if (function_name == "in") { - const ASTIdentifier * ident = args.children.at(0)->as(); - if (!ident || ident->name() != "path") - return false; + if (!isPathNode(node.children.at(0))) + return; - ASTPtr value = args.children.at(1); + auto value = node.children.at(1)->column; + if (!value) + return; - if (value->as()) - { - auto interpreter_subquery = interpretSubquery(value, context, {}, {}); - auto pipeline = interpreter_subquery->execute().pipeline; - SizeLimits limites(context->getSettingsRef().max_rows_in_set, context->getSettingsRef().max_bytes_in_set, OverflowMode::THROW); - Set set(limites, true, context->getSettingsRef().transform_null_in); - set.setHeader(pipeline.getHeader().getColumnsWithTypeAndName()); + const IColumn * column = value.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); - PullingPipelineExecutor executor(pipeline); - Block block; - while (executor.pull(block)) - { - set.insertFromBlock(block.getColumnsWithTypeAndName()); - } - set.finishInsert(); + const ColumnSet * column_set = typeid_cast(column); + if (!column_set) + return; - set.checkColumnsNumber(1); - const auto & set_column = *set.getSetElements()[0]; - for (size_t row = 0; row < set_column.size(); ++row) - res.emplace_back(set_column[row].safeGet(), ZkPathType::Exact); - } - else - { - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + auto set = column_set->getData(); + if (!set->isCreated()) + return; - if (String str; literal->value.tryGet(str)) - { - res.emplace_back(str, ZkPathType::Exact); - } - else if (Tuple tuple; literal->value.tryGet(tuple)) - { - for (auto element : tuple) - res.emplace_back(element.safeGet(), ZkPathType::Exact); - } - else - return false; - } + if (!set->hasExplicitSetElements()) + return; - return true; + set->checkColumnsNumber(1); + auto type = set->getElementsTypes()[0]; + if (!isString(removeNullable(removeLowCardinality(type)))) + return; + + auto values = set->getSetElements()[0]; + size_t size = values->size(); + + for (size_t row = 0; row < size; ++row) + res.emplace_back(values->getDataAt(row).toString(), ZkPathType::Exact); } - else if (function->name == "equals") + else if (function_name == "equals") { - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; + if (!isPathNode(node.children.at(0))) + return; - if (ident->name() != "path") - return false; + auto value = node.children.at(1); + if (!value->column) + return; - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + if (!isString(removeNullable(removeLowCardinality(value->result_type)))) + return; - if (literal->value.getType() != Field::Types::String) - return false; + if (value->column->size() != 1) + return; - res.emplace_back(literal->value.safeGet(), ZkPathType::Exact); - return true; + res.emplace_back(value->column->getDataAt(0).toString(), ZkPathType::Exact); } - else if (allow_unrestricted && function->name == "like") + else if (allow_unrestricted && function_name == "like") { - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; + if (!isPathNode(node.children.at(0))) + return; - if (ident->name() != "path") - return false; + auto value = node.children.at(1); + if (!value->column) + return; - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + if (!isString(removeNullable(removeLowCardinality(value->result_type)))) + return; - if (literal->value.getType() != Field::Types::String) - return false; + if (value->column->size() != 1) + return; - String pattern = literal->value.safeGet(); + String pattern = value->column->getDataAt(0).toString(); bool has_metasymbol = false; String prefix; // pattern prefix before the first metasymbol occurrence for (size_t i = 0; i < pattern.size(); i++) @@ -376,30 +392,28 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, } res.emplace_back(prefix, has_metasymbol ? ZkPathType::Prefix : ZkPathType::Exact); - - return true; } - - return false; } /** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause. */ -static Paths extractPath(const ASTPtr & query, ContextPtr context, bool allow_unrestricted) +static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, ContextPtr context, bool allow_unrestricted) { - const auto & select = query->as(); - if (!select.where()) - return allow_unrestricted ? Paths{{"/", ZkPathType::Recurse}} : Paths(); - Paths res; - return extractPathImpl(*select.where(), res, context, allow_unrestricted) ? res : Paths(); + for (const auto * node : filter_nodes) + extractPathImpl(*node, res, context, allow_unrestricted); + + if (filter_nodes.empty() && allow_unrestricted) + res.emplace_back("/", ZkPathType::Recurse); + + return res; } -void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const { - Paths paths = extractPath(query_info.query, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); + Paths paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); @@ -486,5 +500,26 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c } } +ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, ContextPtr context_) + : SourceStepWithFilter({.header = std::move(header)}) + , storage_limits(query_info.storage_limits) + , context(std::move(context_)) +{ +} + +void ReadFromSystemZooKeeper::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + const auto & header = getOutputStream().header; + MutableColumns res_columns = header.cloneEmptyColumns(); + fillData(res_columns); + + UInt64 num_rows = res_columns.at(0)->size(); + Chunk chunk(std::move(res_columns), num_rows); + + auto source = std::make_shared(header, std::move(chunk)); + source->setStorageLimits(storage_limits); + processors.emplace_back(source); + pipeline.init(Pipe(std::move(source))); +} } diff --git a/src/Storages/System/StorageSystemZooKeeper.h b/src/Storages/System/StorageSystemZooKeeper.h index 20ad29af481..c8988d787a0 100644 --- a/src/Storages/System/StorageSystemZooKeeper.h +++ b/src/Storages/System/StorageSystemZooKeeper.h @@ -11,7 +11,7 @@ class Context; /** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes. */ -class StorageSystemZooKeeper final : public IStorageSystemOneBlock +class StorageSystemZooKeeper final : public IStorage { public: explicit StorageSystemZooKeeper(const StorageID & table_id_); @@ -22,10 +22,19 @@ public: SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; -protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; + void read( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) override; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + bool isSystemStorage() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn(const ASTPtr & node, ContextPtr, const StorageMetadataPtr &) const override; }; } From e8fb13518018e7dad2cc1881375c8578c158a62e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 19:07:53 +0000 Subject: [PATCH 03/52] Reduce memory usage for multiple alter delete mutations. --- src/Interpreters/MutationsInterpreter.cpp | 8 ++- .../02125_many_mutations_2.reference | 4 ++ .../0_stateless/02125_many_mutations_2.sh | 51 +++++++++++++++++++ 3 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02125_many_mutations_2.reference create mode 100755 tests/queries/0_stateless/02125_many_mutations_2.sh diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0b52a1a51bc..26442f94007 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -965,10 +966,15 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s ExpressionActionsChain & actions_chain = stage.expressions_chain; - for (const auto & ast : stage.filters) + if (!stage.filters.empty()) { + auto ast = stage.filters.front(); + if (stage.filters.size() > 1) + ast = makeASTForLogicalAnd(std::move(stage.filters)); + if (!actions_chain.steps.empty()) actions_chain.addStep(); + stage.analyzer->appendExpression(actions_chain, ast, dry_run); stage.filter_column_names.push_back(ast->getColumnName()); } diff --git a/tests/queries/0_stateless/02125_many_mutations_2.reference b/tests/queries/0_stateless/02125_many_mutations_2.reference new file mode 100644 index 00000000000..4bdea51dfc1 --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations_2.reference @@ -0,0 +1,4 @@ +2000 +20000 +0 +1000 diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh new file mode 100755 index 00000000000..df170a402c6 --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" +$CLICKHOUSE_CLIENT -q "insert into many_mutations select number, number + 1 from numbers(2000)" +$CLICKHOUSE_CLIENT -q "system stop merges many_mutations" + +$CLICKHOUSE_CLIENT -q "select count() from many_mutations" + +job() +{ + for i in {1..1000} + do + $CLICKHOUSE_CLIENT -q "alter table many_mutations delete where y = ${i} * 2 settings mutations_sync=0" + done +} + +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & + +wait + +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "system start merges many_mutations" +$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" +$CLICKHOUSE_CLIENT -q "system flush logs" +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "select count() from many_mutations" +$CLICKHOUSE_CLIENT -q "select * from system.part_log where database = currentDatabase() and table == 'many_mutations' and peak_memory_usage > 1e9" From 57255eb56c075c129cbedcc76082c3642560d737 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 19:22:10 +0000 Subject: [PATCH 04/52] Fix equals. --- src/Storages/System/StorageSystemZooKeeper.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index ee33253e40f..c96c8ec60ae 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -333,11 +333,14 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP } else if (function_name == "equals") { - if (!isPathNode(node.children.at(0))) - return; + const ActionsDAG::Node * value = nullptr; - auto value = node.children.at(1); - if (!value->column) + if (isPathNode(node.children.at(0))) + value = node.children.at(1); + else if (isPathNode(node.children.at(1))) + value = node.children.at(0); + + if (!value || !value->column) return; if (!isString(removeNullable(removeLowCardinality(value->result_type)))) From 2e139c21d25f883626e1dbd2a5b8202fabe9297e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Apr 2023 21:57:03 +0000 Subject: [PATCH 05/52] Parallel reading in FROM file() --- src/Processors/ResizeProcessor.h | 2 +- src/Storages/StorageFile.cpp | 15 ++++++++++++--- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index 07d7149ebb4..766c39172a2 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -10,7 +10,7 @@ namespace DB /** Has arbitrary non zero number of inputs and arbitrary non zero number of outputs. * All of them have the same structure. * - * Pulls data from arbitrary input (whenever it is ready) and pushes it to arbitrary output (whenever is is not full). + * Pulls data from arbitrary input (whenever it is ready) and pushes it to arbitrary output (whenever it is not full). * Doesn't do any heavy calculations. * Doesn't preserve an order of data. * diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5fd5664b9e6..84f030ce90e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -40,6 +40,7 @@ #include #include #include +#include "Processors/ResizeProcessor.h" #include #include @@ -700,7 +701,7 @@ Pipe StorageFile::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t num_streams) + const size_t max_num_streams) { if (use_table_fd) { @@ -731,7 +732,8 @@ Pipe StorageFile::read( auto this_ptr = std::static_pointer_cast(shared_from_this()); - if (num_streams > paths.size()) + size_t num_streams = max_num_streams; + if (max_num_streams > paths.size()) num_streams = paths.size(); Pipes pipes; @@ -789,7 +791,14 @@ Pipe StorageFile::read( std::move(read_buffer))); } - return Pipe::unitePipes(std::move(pipes)); + Pipe pipe = Pipe::unitePipes(std::move(pipes)); + /// parallelize output as much as possible + if (num_streams < max_num_streams) + { + pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); + } + + return pipe; } From 96213fa464f31a5d70683f438e2af5442533b742 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Apr 2023 22:17:09 +0000 Subject: [PATCH 06/52] Fix header --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 84f030ce90e..f8b5ea0ced7 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,13 +34,13 @@ #include #include #include +#include #include #include #include #include #include -#include "Processors/ResizeProcessor.h" #include #include From f3e3117d24e99d148f478a454335d7f84c021d8b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 07:41:05 +0000 Subject: [PATCH 07/52] Fix test --- .../02103_with_names_and_types_parallel_parsing.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh index 487282099e2..a6e704093a2 100755 --- a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') DATA_FILE=$USER_FILES_PATH/test_02103.data @@ -14,7 +14,7 @@ FORMATS=('TSVWithNames' 'TSVWithNamesAndTypes' 'TSVRawWithNames' 'TSVRawWithName for format in "${FORMATS[@]}" do $CLICKHOUSE_CLIENT -q "SELECT number, range(number + 10) AS array, toString(number) AS string FROM numbers(10) FORMAT $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') ORDER BY number SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" done rm $DATA_FILE From bea9468285a8df083161ea7c3e4ced027fbd1f5b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 10:47:52 +0000 Subject: [PATCH 08/52] Fix 01548_parallel_parsing_max_memory.sh --- tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh index 8c4900043d0..308b90dcd55 100755 --- a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh +++ b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) yes http://foobarfoobarfoobarfoobarfoobarfoobarfoobar.com | head -c1G > ${CLICKHOUSE_TMP}/1g.csv -$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=100Mi -q "select count() from file('${CLICKHOUSE_TMP}/1g.csv', 'TSV', 'URL String')" +$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=50Mi -q "select count() from file('${CLICKHOUSE_TMP}/1g.csv', 'TSV', 'URL String') settings max_threads=1" From 78038a3c2cec705b405947bc6c0f53e07f844666 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 11:34:04 +0000 Subject: [PATCH 09/52] Fix: do not resize pipeline when there is no files to process (globs expands to empty set) --- src/Storages/StorageFile.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f8b5ea0ced7..665630c3559 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -792,12 +792,13 @@ Pipe StorageFile::read( } Pipe pipe = Pipe::unitePipes(std::move(pipes)); - /// parallelize output as much as possible - if (num_streams < max_num_streams) + /// Parallelize output as much as possible + /// Note: number of streams can be 0 if paths is empty + /// It happens if globs in file(path, ...) expands to empty set i.e. no files to process + if (num_streams > 0 && num_streams < max_num_streams) { pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); } - return pipe; } From 5b2b20a0b06e88ea1165cb59a6ff8f65653cd71f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:13:21 +0200 Subject: [PATCH 10/52] Rename ThreadGroupStatus to ThreadGroup There are methods like getThreadGroup() and ThreadGroupSwitcher class, so seems that this is logical. Signed-off-by: Azat Khuzhin --- src/Common/CurrentThread.cpp | 2 +- src/Common/CurrentThread.h | 6 ++-- src/Common/ThreadStatus.cpp | 6 ++-- src/Common/ThreadStatus.h | 24 +++++++-------- src/Interpreters/Aggregator.cpp | 4 +-- src/Interpreters/ExternalLoader.cpp | 2 +- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/ProcessList.h | 4 +-- src/Interpreters/ThreadStatusExt.cpp | 30 +++++++++---------- .../Executors/CompletedPipelineExecutor.cpp | 2 +- .../PullingAsyncPipelineExecutor.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Impl/ParallelFormattingOutputFormat.cpp | 4 +-- .../Impl/ParallelFormattingOutputFormat.h | 4 +-- .../Impl/ParallelParsingInputFormat.cpp | 4 +-- .../Formats/Impl/ParallelParsingInputFormat.h | 4 +-- .../Transforms/buildPushingToViewsChain.cpp | 6 ++-- src/Storages/MergeTree/MergeList.cpp | 4 +-- src/Storages/MergeTree/MergeList.h | 6 ++-- 19 files changed, 59 insertions(+), 59 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 6ec46d6508c..fd2ad0bbaf1 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -90,7 +90,7 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, @@ -69,9 +69,9 @@ public: /// You must call one of these methods when create a query child thread: /// Add current thread to a group associated with the thread group - static void attachToGroup(const ThreadGroupStatusPtr & thread_group); + static void attachToGroup(const ThreadGroupPtr & thread_group); /// Is useful for a ThreadPool tasks - static void attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group); + static void attachToGroupIfDetached(const ThreadGroupPtr & thread_group); /// Non-master threads call this method in destructor automatically static void detachFromGroupIfNotDetached(); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1b783aa9ec4..e00c9b168a9 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -61,7 +61,7 @@ static thread_local ThreadStack alt_stack; static thread_local bool has_alt_stack = false; #endif -ThreadGroupStatus::ThreadGroupStatus() +ThreadGroup::ThreadGroup() : master_thread_id(CurrentThread::get().thread_id) {} @@ -119,7 +119,7 @@ ThreadStatus::ThreadStatus() #endif } -ThreadGroupStatusPtr ThreadStatus::getThreadGroup() const +ThreadGroupPtr ThreadStatus::getThreadGroup() const { return thread_group; } @@ -139,7 +139,7 @@ ContextPtr ThreadStatus::getGlobalContext() const return global_context.lock(); } -void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) +void ThreadGroup::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) { std::lock_guard lock(mutex); shared_data.logs_queue_ptr = logs_queue; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index cb85aa67b11..16083fe0925 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -58,15 +58,15 @@ using ThreadStatusPtr = ThreadStatus *; * Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks). * Use via CurrentThread::getGroup. */ -class ThreadGroupStatus; -using ThreadGroupStatusPtr = std::shared_ptr; +class ThreadGroup; +using ThreadGroupPtr = std::shared_ptr; -class ThreadGroupStatus +class ThreadGroup { public: - ThreadGroupStatus(); + ThreadGroup(); using FatalErrorCallback = std::function; - ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); /// The first thread created this thread group const UInt64 master_thread_id; @@ -104,9 +104,9 @@ public: void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); /// When new query starts, new thread group is created for it, current thread becomes master thread of the query - static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + static ThreadGroupPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); - static ThreadGroupStatusPtr createForBackgroundProcess(ContextPtr storage_context); + static ThreadGroupPtr createForBackgroundProcess(ContextPtr storage_context); std::vector getInvolvedThreadIds() const; void linkThread(UInt64 thread_it); @@ -163,7 +163,7 @@ public: private: /// Group of threads, to which this thread attached - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; /// Is set once ContextWeakPtr global_context; @@ -174,7 +174,7 @@ private: using FatalErrorCallback = std::function; FatalErrorCallback fatal_error_callback; - ThreadGroupStatus::SharedData local_data; + ThreadGroup::SharedData local_data; bool performance_counters_finalized = false; @@ -215,7 +215,7 @@ public: ThreadStatus(); ~ThreadStatus(); - ThreadGroupStatusPtr getThreadGroup() const; + ThreadGroupPtr getThreadGroup() const; const String & getQueryId() const; @@ -239,7 +239,7 @@ public: void setInternalThread(); /// Attaches slave thread to existing thread group - void attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached = true); /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachFromGroup(); @@ -287,7 +287,7 @@ private: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database); - void attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_); + void attachToGroupImpl(const ThreadGroupPtr & thread_group_); }; /** diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d6fbf072d05..2d5de796e1c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2315,7 +2315,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( std::atomic next_bucket_to_merge = 0; - auto converter = [&](size_t thread_id, ThreadGroupStatusPtr thread_group) + auto converter = [&](size_t thread_id, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -3043,7 +3043,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 04a116ec0c7..c1cbd8b75be 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -967,7 +967,7 @@ private: } /// Does the loading, possibly in the separate thread. - void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupStatusPtr thread_group = {}) + void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupPtr thread_group = {}) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 51053bd2884..aca474bf152 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -340,7 +340,7 @@ QueryStatus::QueryStatus( const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds) : WithContext(context_) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index a04beac4901..b593bcef395 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -86,7 +86,7 @@ protected: ClientInfo client_info; /// Info about all threads involved in query execution - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; Stopwatch watch; @@ -162,7 +162,7 @@ public: const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 72ce08bf653..070cd3f98e1 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -41,14 +41,14 @@ namespace ErrorCodes extern const int CANNOT_SET_THREAD_PRIORITY; } -ThreadGroupStatus::ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) +ThreadGroup::ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) : master_thread_id(CurrentThread::get().thread_id) , query_context(query_context_) , global_context(query_context_->getGlobalContext()) , fatal_error_callback(fatal_error_callback_) {} -std::vector ThreadGroupStatus::getInvolvedThreadIds() const +std::vector ThreadGroup::getInvolvedThreadIds() const { std::vector res; @@ -60,22 +60,22 @@ std::vector ThreadGroupStatus::getInvolvedThreadIds() const return res; } -void ThreadGroupStatus::linkThread(UInt64 thread_it) +void ThreadGroup::linkThread(UInt64 thread_it) { std::lock_guard lock(mutex); thread_ids.insert(thread_it); } -ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) +ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { - auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); + auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); group->memory_tracker.setDescription("(for query)"); return group; } -ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr storage_context) +ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_context) { - auto group = std::make_shared(storage_context); + auto group = std::make_shared(storage_context); group->memory_tracker.setDescription("background process to apply mutate/merge in table"); /// However settings from storage context have to be applied @@ -89,7 +89,7 @@ ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr st return group; } -void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash) +void ThreadGroup::attachQueryForLog(const String & query_, UInt64 normalized_hash) { auto hash = normalized_hash ? normalized_hash : normalizedQueryHash(query_); @@ -109,7 +109,7 @@ void ThreadStatus::attachQueryForLog(const String & query_) thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash); } -void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { std::lock_guard lock(mutex); shared_data.profile_queue_ptr = profile_queue; @@ -168,7 +168,7 @@ void ThreadStatus::applyQuerySettings() #endif } -void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_) +void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) { /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; @@ -234,7 +234,7 @@ void ThreadStatus::setInternalThread() internal_thread = true; } -void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached) +void ThreadStatus::attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached) { if (thread_group && check_detached) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached"); @@ -541,14 +541,14 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) views_log->add(element); } -void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroup(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; current_thread->attachToGroup(thread_group, true); } -void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroupIfDetached(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; @@ -574,7 +574,7 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::func if (!query_context->hasQueryContext()) query_context->makeQueryContext(); - auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); + auto group = ThreadGroup::createForQuery(query_context, std::move(fatal_error_callback)); CurrentThread::attachToGroup(group); } @@ -584,7 +584,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::functiongetPort(IOutputFormat::PortKind::Main).getHeader(); } -static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 3aec7608e6d..ac40cef35d9 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -97,7 +97,7 @@ struct PushingAsyncPipelineExecutor::Data } }; -static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index 3fc57ca1c1e..62ee4e4a48d 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -96,7 +96,7 @@ namespace DB } - void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -157,7 +157,7 @@ namespace DB } - void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 790d05e83dd..4e5aaab5dcb 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -270,10 +270,10 @@ private: } /// Collects all temporary buffers into main WriteBuffer. - void collectorThreadFunction(const ThreadGroupStatusPtr & thread_group); + void collectorThreadFunction(const ThreadGroupPtr & thread_group); /// This function is executed in ThreadPool and the only purpose of it is to format one Chunk into a continuous buffer in memory. - void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group); + void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group); void setRowsBeforeLimit(size_t rows_before_limit) override { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index a2e5074efb1..f4d619a263b 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -8,7 +8,7 @@ namespace DB { -void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr thread_group) +void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -62,7 +62,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr } } -void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number) +void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 97df9308dbf..ae9f123d411 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -317,8 +317,8 @@ private: } } - void segmentatorThreadFunction(ThreadGroupStatusPtr thread_group); - void parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number); + void segmentatorThreadFunction(ThreadGroupPtr thread_group); + void parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number); /// Save/log a background exception, set termination flag, wake up all /// threads. This function is used by segmentator and parsed threads. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 91845bc18ad..e32e2116f71 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -270,14 +270,14 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroupStatus always should have context attached, + /// NOTE: ThreadGroup always should have context attached, /// otherwise entry to the system.query_views_log will not be added /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupStatusPtr running_group; + ThreadGroupPtr running_group; if (current_thread) running_group = current_thread->getThreadGroup(); if (!running_group) - running_group = std::make_shared(context); + running_group = std::make_shared(context); /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 7970efd1a20..91e1cb1b078 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,7 +11,7 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group) +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) { chassert(thread_group); @@ -59,7 +59,7 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } - thread_group = ThreadGroupStatus::createForBackgroundProcess(context); + thread_group = ThreadGroup::createForBackgroundProcess(context); } MergeInfo MergeListElement::getInfo() const diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index c3ba4f00f7f..308f00feda9 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -69,11 +69,11 @@ struct Settings; class ThreadGroupSwitcher : private boost::noncopyable { public: - explicit ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group); + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); ~ThreadGroupSwitcher(); private: - ThreadGroupStatusPtr prev_thread_group; + ThreadGroupPtr prev_thread_group; }; struct MergeListElement : boost::noncopyable @@ -113,7 +113,7 @@ struct MergeListElement : boost::noncopyable /// Detected after merge already started std::atomic merge_algorithm; - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; MergeListElement( const StorageID & table_id_, From aacf2a083882bf395d94a6f7dba1fa2d35db2991 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:15:10 +0200 Subject: [PATCH 11/52] Move ThreadGroupSwitcher to ThreadStatus.h (out from MergeTree code) Signed-off-by: Azat Khuzhin --- src/Common/ThreadStatus.h | 16 +++++++++++++++- src/Interpreters/ThreadStatusExt.cpp | 18 ++++++++++++++++++ src/Storages/MergeTree/MergeList.cpp | 18 ------------------ src/Storages/MergeTree/MergeList.h | 14 -------------- 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 16083fe0925..600dfc56d2b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -41,7 +41,6 @@ class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; -class ThreadGroupSwitcher; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -120,6 +119,21 @@ private: std::unordered_set thread_ids; }; +/** + * Since merge is executed with multiple threads, this class + * switches the parent MemoryTracker as part of the thread group to account all the memory used. + */ +class ThreadGroupSwitcher : private boost::noncopyable +{ +public: + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); + ~ThreadGroupSwitcher(); + +private: + ThreadGroupPtr prev_thread_group; +}; + + /** * We use **constinit** here to tell the compiler the current_thread variable is initialized. * If we didn't help the compiler, then it would most likely add a check before every use of the variable to initialize it if needed. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 070cd3f98e1..c6e36263e7c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -115,6 +115,24 @@ void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQu shared_data.profile_queue_ptr = profile_queue; } +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) +{ + chassert(thread_group); + + /// might be nullptr + prev_thread_group = CurrentThread::getGroup(); + + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); +} + +ThreadGroupSwitcher::~ThreadGroupSwitcher() +{ + CurrentThread::detachFromGroupIfNotDetached(); + if (prev_thread_group) + CurrentThread::attachToGroup(prev_thread_group); +} + void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { if (!thread_group) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 91e1cb1b078..0bf662921ad 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,24 +11,6 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) -{ - chassert(thread_group); - - /// might be nullptr - prev_thread_group = CurrentThread::getGroup(); - - CurrentThread::detachFromGroupIfNotDetached(); - CurrentThread::attachToGroup(thread_group); -} - -ThreadGroupSwitcher::~ThreadGroupSwitcher() -{ - CurrentThread::detachFromGroupIfNotDetached(); - if (prev_thread_group) - CurrentThread::attachToGroup(prev_thread_group); -} - MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 308f00feda9..9c8c2ebd1e4 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -62,20 +62,6 @@ using MergeListEntry = BackgroundProcessListEntry; struct Settings; -/** - * Since merge is executed with multiple threads, this class - * switches the parent MemoryTracker as part of the thread group to account all the memory used. - */ -class ThreadGroupSwitcher : private boost::noncopyable -{ -public: - explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); - ~ThreadGroupSwitcher(); - -private: - ThreadGroupPtr prev_thread_group; -}; - struct MergeListElement : boost::noncopyable { const StorageID table_id; From 44ae8485f185ba43323d61a2e85007f5067b4e2b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:29:51 +0200 Subject: [PATCH 12/52] Use one ThreadGroup while pushing to materialized views Before this patch only the case when ThreadStatus (current_thread) already has ThreadGroup works that way, after they will be identical in this aspect. But this should not affect anything, but it just make sense. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterInsertQuery.cpp | 24 ++++++++++++++++--- src/Interpreters/InterpreterInsertQuery.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 22 ++++++++--------- .../Transforms/buildPushingToViewsChain.h | 4 ++++ 4 files changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b4a19ea7403..e78a61831a1 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -233,8 +234,14 @@ Chain InterpreterInsertQuery::buildChain( ThreadStatusesHolderPtr thread_status_holder, std::atomic_uint64_t * elapsed_counter_ms) { + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + auto sample = getSampleBlock(columns, table, metadata_snapshot); - return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, elapsed_counter_ms); + return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, running_group, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( @@ -242,6 +249,7 @@ Chain InterpreterInsertQuery::buildChainImpl( const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms) { ThreadStatus * thread_status = current_thread; @@ -273,7 +281,9 @@ Chain InterpreterInsertQuery::buildChainImpl( } else { - out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status_holder, elapsed_counter_ms); + out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, + query_ptr, no_destination, + thread_status_holder, running_group, elapsed_counter_ms); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. @@ -461,9 +471,17 @@ BlockIO InterpreterInsertQuery::execute() pipeline = interpreter_watch.buildQueryPipeline(); } + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); for (size_t i = 0; i < out_streams_size; ++i) { - auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); + auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, + /* thread_status_holder= */ nullptr, + running_group, + /* elapsed_counter_ms= */ nullptr); out_chains.emplace_back(std::move(out)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index bb6509a9102..f60d6567d74 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -70,6 +70,7 @@ private: const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms); }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e32e2116f71..0bdd7a88851 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -195,6 +195,7 @@ Chain buildPushingToViewsChain( const ASTPtr & query_ptr, bool no_destination, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms, const Block & live_view_header) { @@ -270,15 +271,6 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroup always should have context attached, - /// otherwise entry to the system.query_views_log will not be added - /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(context); - /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls /// Later on, before doing any task related to a view, we'll switch to its ThreadStatus, do the work, @@ -354,18 +346,24 @@ Chain buildPushingToViewsChain( runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms, storage_header); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms, storage_header); } else if (auto * window_view = dynamic_cast(view.get())) { runtime_stats->type = QueryViewsLogElement::ViewType::WINDOW; query = window_view->getMergeableQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms); } else out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), false, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ false, + thread_status_holder, running_group, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 76131a8df6e..a2e7e39ff23 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -61,6 +61,10 @@ Chain buildPushingToViewsChain( /// We could specify separate thread_status for each view. /// Needed mainly to collect counters separately. Should be improved. ThreadStatusesHolderPtr thread_status_holder, + /// Usually current_thread->getThreadGroup(), but sometimes ThreadStatus + /// may not have ThreadGroup (i.e. Buffer background flush), and in this + /// case it should be passed outside. + ThreadGroupPtr running_group, /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, /// LiveView executes query itself, it needs source block structure. From d0a37f8798a2c87b37e31a4fa4f1b656b11d5982 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Sat, 8 Apr 2023 01:44:43 +0800 Subject: [PATCH 13/52] fix test test_multiple_disks --- tests/integration/test_multiple_disks/test.py | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 221d830f62e..f6e5d456dfb 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -90,6 +90,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external", @@ -100,6 +102,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external_no_merges", @@ -110,6 +114,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external_no_merges", @@ -120,6 +126,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 1, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "one_more_small_jbod_with_external", @@ -130,6 +138,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "one_more_small_jbod_with_external", @@ -140,6 +150,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "jbods_with_external", @@ -150,6 +162,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "10485760", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "jbods_with_external", @@ -160,6 +174,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "moving_jbod_with_external", @@ -170,6 +186,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.7, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "moving_jbod_with_external", @@ -180,6 +198,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.7, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "default_disk_with_external", @@ -190,6 +210,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "2097152", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "default_disk_with_external", @@ -200,6 +222,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "20971520", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -210,6 +234,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -220,6 +246,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -230,6 +258,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "1024", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -240,6 +270,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "1024000000", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, ] From 8fdc2b33260ef8c02ae79b4d5d98e83df3f233e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 20:06:11 +0000 Subject: [PATCH 14/52] Perf test --- tests/performance/reading_from_file.xml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/performance/reading_from_file.xml diff --git a/tests/performance/reading_from_file.xml b/tests/performance/reading_from_file.xml new file mode 100644 index 00000000000..992f1e8acae --- /dev/null +++ b/tests/performance/reading_from_file.xml @@ -0,0 +1,9 @@ + + +INSERT INTO function file(reading_from_file.parquet) SELECT URL FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert=1 + +SELECT sum(length(base58Encode(URL))) FROM file(reading_from_file.parquet) FORMAT Null + +INSERT INTO FUNCTION file(reading_from_file.parquet) SELECT * FROM numbers(0) SETTINGS engine_file_truncate_on_insert=1 + + From bf3a08dc98842288d1b433efcfd614103fc8e1cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 22:58:08 +0200 Subject: [PATCH 15/52] Fix 02477_projection_materialize_and_zero_copy flakienss (due to index granularity randomization) Signed-off-by: Azat Khuzhin --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index d4c24b31da2..1845919890c 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 -SETTINGS allow_remote_fs_zero_copy_replication=1; +SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8092, index_granularity_bytes='10Mi'; insert into t (c1, c18) select number, -number from numbers(2000000); From 0dc3193e6bf3a29a50242b1743ef1a652570adcc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 17:29:08 +0000 Subject: [PATCH 16/52] Fix 02293_formats_json_columns --- .../02293_formats_json_columns.reference | 14 +++++++------- .../0_stateless/02293_formats_json_columns.sh | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02293_formats_json_columns.reference b/tests/queries/0_stateless/02293_formats_json_columns.reference index f59f02ad42b..d3fb006a73f 100644 --- a/tests/queries/0_stateless/02293_formats_json_columns.reference +++ b/tests/queries/0_stateless/02293_formats_json_columns.reference @@ -86,18 +86,18 @@ d Nullable(String) \N \N 3 \N \N \N \N String OK -3 -2 1 +2 +3 c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(String) 1 1 \N -2 2 \N -3 3 \N -1 \N \N -2 \N \N -3 \N \N 1 2 String +1 \N \N +2 2 \N +2 \N \N +3 3 \N +3 \N \N OK OK diff --git a/tests/queries/0_stateless/02293_formats_json_columns.sh b/tests/queries/0_stateless/02293_formats_json_columns.sh index 7a21f8d9bab..0aaf2abfc45 100755 --- a/tests/queries/0_stateless/02293_formats_json_columns.sh +++ b/tests/queries/0_stateless/02293_formats_json_columns.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') DATA_FILE=$USER_FILES_PATH/data_02293 @@ -17,13 +17,13 @@ echo "JSONColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONColumns" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns)" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns) order by a" echo "JSONCompactColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONCompactColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONCompactColumns" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns)" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns) order by c1, c2, c3" echo "JSONColumnsWithMetadata" $CLICKHOUSE_CLIENT -q "select sum(a) as sum, avg(a) as avg from test_02293 group by a % 4 with totals order by tuple(sum, avg) format JSONColumnsWithMetadata" --extremes=1 | grep -v "elapsed" @@ -49,9 +49,9 @@ echo ' ' > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') settings input_format_skip_unknown_fields=0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') settings input_format_skip_unknown_fields=1" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns) order by b, a, c, d" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') order by a, t settings input_format_skip_unknown_fields=0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') order by a, t settings input_format_skip_unknown_fields=1" echo ' [ @@ -75,8 +75,8 @@ echo ' ' > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns, 'a UInt32, t UInt32')" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns) order by c1, c2, c3" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns, 'a UInt32, t UInt32') order by a, t" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' echo ' { From 32ac23803f2904187b0a30cdbe7529012d1b12cf Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 19:02:18 +0000 Subject: [PATCH 17/52] Fix test which can become flaky due to file() parallelization --- ...decompression_with_escape_sequence_at_the_end_of_buffer.sh | 4 ++-- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 2 +- tests/queries/0_stateless/02211_jsonl_format_extension.sql | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh index 47cf6e06b48..2f8d8f06dee 100755 --- a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh +++ b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh @@ -15,8 +15,8 @@ ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " set min_chunk_bytes_for_parallel_parsing=10485760; set max_read_buffer_size = 65536; set input_format_parallel_parsing = 0; -select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null; +select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') order by foo limit 30 format Null; set input_format_parallel_parsing = 1; -select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null; +select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') order by foo limit 30 format Null; " diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index dfdc71e0f0b..22d6d2938cd 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -28,5 +28,5 @@ function cleanup() trap cleanup EXIT ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; -${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String')"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String') order by a"; diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index 08fff5a11f5..907cdd5cf92 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); -select * from file('data.jsonl'); +select * from file('data.jsonl') order by x; From 0fbb05bf55ce84d1f998b33816f9a21c22456386 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 19:15:18 +0000 Subject: [PATCH 18/52] Fix 02286_mysql_dump_input_format --- .../02286_mysql_dump_input_format.reference | 2 +- .../02286_mysql_dump_input_format.sh | 92 +++++++++---------- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference index a736358b9b7..25a78651e33 100644 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference @@ -12,13 +12,13 @@ 4 \N 5 \N 6 7 +7 6 \N 1 \N 2 \N 3 \N 3 \N 4 \N 5 -7 6 OK 1 2 diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh index 891734e9ad3..1139c1ea68c 100755 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh @@ -9,140 +9,140 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex cp $CURDIR/data_mysql_dump/dump*.sql $USER_FILES_PATH -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), y Nullable(Int32)')" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'a Nullable(Int32), b Nullable(Int32)') settings input_format_mysql_dump_map_column_names = 0" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'y Nullable(Int32), x Nullable(Int32)') settings input_format_mysql_dump_map_column_names = 1" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') settings input_format_skip_unknown_fields = 0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') settings input_format_skip_unknown_fields = 1" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), y Nullable(Int32)') order by x, y" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'a Nullable(Int32), b Nullable(Int32)') order by a, b settings input_format_mysql_dump_map_column_names = 0" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'y Nullable(Int32), x Nullable(Int32)') order by y, x settings input_format_mysql_dump_map_column_names = 1" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') order by x, z settings input_format_skip_unknown_fields = 0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') order by x, z settings input_format_skip_unknown_fields = 1" echo "dump1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32)') settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'EMPTY_DATA_PASSED' && echo 'OK' || echo 'FAIL' echo "dump2" $CLICKHOUSE_CLIENT -q "desc file(dump2.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump3" $CLICKHOUSE_CLIENT -q "desc file(dump3.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" echo "dump4" $CLICKHOUSE_CLIENT -q "desc file(dump4.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump5" $CLICKHOUSE_CLIENT -q "desc file(dump5.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump6" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump7" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump8" $CLICKHOUSE_CLIENT -q "desc file(dump8.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump8.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump8.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump8.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump9" $CLICKHOUSE_CLIENT -q "desc file(dump9.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump10" $CLICKHOUSE_CLIENT -q "desc file(dump10.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump11" $CLICKHOUSE_CLIENT -q "desc file(dump11.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump12" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump13" $CLICKHOUSE_CLIENT -q "desc file(dump13.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits'" -$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits'" +$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits', max_threads=1" echo "dump14" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump15" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" rm $USER_FILES_PATH/dump*.sql From bcb913e7b362191f36895617fbb97b923264c9c7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 20:13:24 +0000 Subject: [PATCH 19/52] A try to prevent possible tests flakyness due to file() parallelization --- .../0_stateless/02293_arrow_dictionary_indexes.sql | 2 +- .../0_stateless/02314_csv_tsv_skip_first_lines.sql | 4 ++-- .../queries/0_stateless/02383_arrow_dict_special_cases.sh | 8 ++++---- .../0_stateless/02455_one_row_from_csv_memory_usage.sh | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql index 3ff6a5ffbb3..ec68d1a4443 100644 --- a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql +++ b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest insert into function file(02293_data.arrow) select toLowCardinality(toString(number)) from numbers(300) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1; -select * from file(02293_data.arrow); +select * from file(02293_data.arrow) settings max_threads=1; diff --git a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql index 4a0cef35310..a569b6e9b84 100644 --- a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql +++ b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql @@ -3,10 +3,10 @@ insert into function file(data_02314.csv) select number, number + 1 from numbers(5) settings engine_file_truncate_on_insert=1; insert into function file(data_02314.csv) select number, number + 1, number + 2 from numbers(5); desc file(data_02314.csv) settings input_format_csv_skip_first_lines=5; -select * from file(data_02314.csv) settings input_format_csv_skip_first_lines=5; +select * from file(data_02314.csv) order by c1 settings input_format_csv_skip_first_lines=5; insert into function file(data_02314.tsv) select number, number + 1 from numbers(5) settings engine_file_truncate_on_insert=1; insert into function file(data_02314.tsv) select number, number + 1, number + 2 from numbers(5); desc file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; -select * from file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; +select * from file(data_02314.tsv) order by c1 settings input_format_tsv_skip_first_lines=5; diff --git a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh index 86f95873f14..bab0b57f116 100755 --- a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh +++ b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh @@ -13,17 +13,17 @@ cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/ cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/ $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')" $CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL $CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow') settings max_threads=1" rm -rf $USER_FILES_PATH/test_02383 diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 7506e78455d..9076d7a8a5f 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -7,5 +7,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $user_files_path/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') LIMIT 1 settings max_memory_usage=100000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" From d80364ffa0bebd86e25dcc89bf3be325da26830a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 22:51:12 +0000 Subject: [PATCH 20/52] Fix tests: truncate output file on insert --- tests/queries/0_stateless/02187_msg_pack_uuid.sh | 6 +++--- tests/queries/0_stateless/02211_jsonl_format_extension.sql | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02187_msg_pack_uuid.sh b/tests/queries/0_stateless/02187_msg_pack_uuid.sh index 9be92d66790..f04ef09a8c8 100755 --- a/tests/queries/0_stateless/02187_msg_pack_uuid.sh +++ b/tests/queries/0_stateless/02187_msg_pack_uuid.sh @@ -5,13 +5,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_str.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='str'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_str.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='str', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_str.msgpack', 'MsgPack', 'uuid UUID')" -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='bin'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='bin', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID')" -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='ext'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='ext', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID')" $CLICKHOUSE_CLIENT -q "select c1, toTypeName(c1) from file('uuid_ext.msgpack') settings input_format_msgpack_number_of_columns=1" diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index 907cdd5cf92..a95b19f4e3b 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest -insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); +insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10) SETTINGS engine_file_truncate_on_insert=1; select * from file('data.jsonl') order by x; From 09ea79aaf7955a9db36126e9a5f18e93b98704ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 03:04:26 +0200 Subject: [PATCH 21/52] Add support for {server_uuid} macro --- programs/keeper/Keeper.cpp | 1 - programs/server/Server.cpp | 2 +- src/Common/Exception.h | 1 - src/Common/Macros.cpp | 12 +++++++++++- src/Daemon/BaseDaemon.h | 1 - src/Dictionaries/MongoDBDictionarySource.cpp | 1 - src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 9 +++------ src/IO/HTTPCommon.cpp | 2 -- src/IO/ReadWriteBufferFromHTTP.h | 1 - src/Processors/Sources/MongoDBSource.cpp | 1 - src/Storages/StorageMongoDB.cpp | 1 - 11 files changed, 15 insertions(+), 17 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 266b363eb47..3853c955171 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9ef9f704f61..164e1ce14e5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -981,7 +981,7 @@ try StatusFile status{path / "status", StatusFile::write_full_info}; - DB::ServerUUID::load(path / "uuid", log); + ServerUUID::load(path / "uuid", log); /// Try to increase limit on number of open files. { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 8e50c1114f4..170e0d32b3c 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -4,7 +4,6 @@ #include #include -#include #include #include diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index e5d4be446c1..18e65bbaede 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -1,8 +1,9 @@ #include #include #include -#include #include +#include +#include namespace DB @@ -105,6 +106,15 @@ String Macros::expand(const String & s, res += toString(info.table_id.uuid); info.expanded_uuid = true; } + else if (macro_name == "server_uuid") + { + auto uuid = ServerUUID::get(); + if (UUIDHelpers::Nil == uuid) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Macro {server_uuid} expanded to zero, which means the UUID is not initialized (most likely it's not a server application)"); + res += toString(uuid); + info.expanded_other = true; + } else if (info.shard && macro_name == "shard") { res += *info.shard; diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index d28f9403c16..f90f403364b 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index a9555a94304..922e1e71bbb 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -67,7 +67,6 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) #include #include #include -#include // only after poco // naming conflict: diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 1c192a0d89c..70f39d893f7 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -8,7 +8,6 @@ #if USE_AWS_S3 -#include #include #include @@ -19,9 +18,7 @@ #include #include #include -#include -#include #include #include @@ -87,10 +84,10 @@ public: private: static String getServerUUID() { - DB::UUID server_uuid = DB::ServerUUID::get(); - if (server_uuid == DB::UUIDHelpers::Nil) + UUID server_uuid = ServerUUID::get(); + if (server_uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::LOGICAL_ERROR, "Server UUID is not initialized"); - return DB::toString(server_uuid); + return toString(server_uuid); } }; diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 551ce797757..9eabe3573f4 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -7,8 +7,6 @@ #include #include -#include - #include "config.h" #if USE_SSL diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 784110f735e..28f7a7f600b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -21,7 +21,6 @@ #include #include #include -#include #include #include #include "config.h" diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index a8bfefdf8a6..9e50cdcf6ab 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -15,7 +15,6 @@ #include #include #include -#include // only after poco // naming conflict: diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 0bad4ab8bf0..63b8c2d00a1 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From 6292a08c9c98dd85d28e5a7f10759a858c752421 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 03:25:38 +0200 Subject: [PATCH 22/52] Add a test --- src/Common/Macros.cpp | 8 +++++--- .../0_stateless/02711_server_uuid_macro.reference | 1 + .../0_stateless/02711_server_uuid_macro.sql | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02711_server_uuid_macro.reference create mode 100644 tests/queries/0_stateless/02711_server_uuid_macro.sql diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 18e65bbaede..f43fed6c499 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -12,6 +12,8 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NO_ELEMENTS_IN_CONFIG; } Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log) @@ -96,7 +98,7 @@ String Macros::expand(const String & s, else if (macro_name == "uuid" && !info.expand_special_macros_only) { if (info.table_id.uuid == UUIDHelpers::Nil) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Macro 'uuid' and empty arguments of ReplicatedMergeTree " + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Macro 'uuid' and empty arguments of ReplicatedMergeTree " "are supported only for ON CLUSTER queries with Atomic database engine"); /// For ON CLUSTER queries we don't want to require all macros definitions in initiator's config. /// However, initiator must check that for cross-replication cluster zookeeper_path does not contain {uuid} macro. @@ -135,7 +137,7 @@ String Macros::expand(const String & s, info.has_unknown = true; } else - throw Exception(ErrorCodes::SYNTAX_ERROR, "No macro '{}' in config while processing substitutions in " + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No macro '{}' in config while processing substitutions in " "'{}' at '{}' or macro is not supported here", macro_name, s, toString(begin)); pos = end + 1; @@ -152,7 +154,7 @@ String Macros::getValue(const String & key) const { if (auto it = macros.find(key); it != macros.end()) return it->second; - throw Exception(ErrorCodes::SYNTAX_ERROR, "No macro {} in config", key); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No macro {} in config", key); } diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.reference b/tests/queries/0_stateless/02711_server_uuid_macro.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02711_server_uuid_macro.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.sql b/tests/queries/0_stateless/02711_server_uuid_macro.sql new file mode 100644 index 00000000000..f708156c0ae --- /dev/null +++ b/tests/queries/0_stateless/02711_server_uuid_macro.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test; + +-- You can create a table with the {server_uuid} substituted. +CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; + +-- The server UUID is correctly substituted. +SELECT engine_full LIKE ('%replica-' || serverUUID()::String || '%') FROM system.tables WHERE database = currentDatabase() AND name = 'test'; + +-- An attempt to create a second table with the same UUID results in error. +CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } + +-- The macro {server_uuid} is special, not a configuration-type macro. It's normal that it is inaccessible with the getMacro function. +SELECT getMacro('server_uuid'); -- { serverError NO_ELEMENTS_IN_CONFIG } + +DROP TABLE test NO DELAY; From d23cd018ec8196184a6a275004559202bef4b9ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 07:46:27 +0200 Subject: [PATCH 23/52] Update test --- .../0_stateless/01148_zookeeper_path_macros_unfolding.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql index 7234cee96e0..505c406c2cc 100644 --- a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -12,7 +12,7 @@ DETACH TABLE rmt1; ATTACH TABLE rmt1; SHOW CREATE TABLE rmt1; -CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 } +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 36 } CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n; SHOW CREATE TABLE rmt; RENAME TABLE rmt TO rmt2; -- { serverError 48 } @@ -24,7 +24,7 @@ SET distributed_ddl_output_mode='none'; DROP DATABASE IF EXISTS test_01148_atomic; CREATE DATABASE test_01148_atomic ENGINE=Atomic; CREATE TABLE test_01148_atomic.rmt2 ON CLUSTER test_shard_localhost (n int, PRIMARY KEY n) ENGINE=ReplicatedMergeTree; -CREATE TABLE test_01148_atomic.rmt3 AS test_01148_atomic.rmt2; -- { serverError 62 } +CREATE TABLE test_01148_atomic.rmt3 AS test_01148_atomic.rmt2; -- { serverError 36 } CREATE TABLE test_01148_atomic.rmt4 ON CLUSTER test_shard_localhost AS test_01148_atomic.rmt2; SHOW CREATE TABLE test_01148_atomic.rmt2; RENAME TABLE test_01148_atomic.rmt4 to test_01148_atomic.rmt3; From c3c07c05ea23e8f9f175c76cc7accbcdf14b50f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 09:36:20 +0300 Subject: [PATCH 24/52] Update 02207_allow_plaintext_and_no_password.sh --- .../0_stateless/02207_allow_plaintext_and_no_password.sh | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh index 693f1d817e3..0345a0e6394 100755 --- a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh +++ b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh @@ -1,11 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-fasttest -# Tag no-tsan: requires jemalloc to track small allocations -# Tag no-asan: requires jemalloc to track small allocations -# Tag no-ubsan: requires jemalloc to track small allocations -# Tag no-msan: requires jemalloc to track small allocations - - +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 3bc5c6423b2d522418e553f8050947343dbf6160 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sun, 9 Apr 2023 16:54:21 +0200 Subject: [PATCH 25/52] Update formats.md --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index de8357345db..5c59d8d74ca 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -78,7 +78,7 @@ The supported formats are: | [Null](#null) | ✗ | ✔ | | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✔ | -| [LineAsString](#lineasstring) | ✔ | ✗ | +| [LineAsString](#lineasstring) | ✔ | ✔ | | [Regexp](#data-format-regexp) | ✔ | ✗ | | [RawBLOB](#rawblob) | ✔ | ✔ | | [MsgPack](#msgpack) | ✔ | ✔ | From 286424be05c6734543b5157287d5359b80dbe385 Mon Sep 17 00:00:00 2001 From: caipengxiang <291458254@qq.com> Date: Sun, 9 Apr 2023 22:59:17 +0800 Subject: [PATCH 26/52] bugfix: compare Bits and sizeof(Arithmetic) * 8 --- base/base/wide_integer_impl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 7cdb527f9cf..ed4570d5e3f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -155,13 +155,13 @@ struct common_type, Arithmetic> std::is_floating_point_v, Arithmetic, std::conditional_t< - sizeof(Arithmetic) < Bits * sizeof(long), + sizeof(Arithmetic) * 8 < Bits, wide::integer, std::conditional_t< - Bits * sizeof(long) < sizeof(Arithmetic), + Bits < sizeof(Arithmetic) * 8, Arithmetic, std::conditional_t< - Bits * sizeof(long) == sizeof(Arithmetic) && (std::is_same_v || std::is_signed_v), + Bits == sizeof(Arithmetic) * 8 && (std::is_same_v || std::is_signed_v), Arithmetic, wide::integer>>>>; }; From 44e418987cdf12ad54bb9fb220947c9451de88b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 22:26:49 +0300 Subject: [PATCH 27/52] Update tests/queries/0_stateless/02711_server_uuid_macro.sql Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- tests/queries/0_stateless/02711_server_uuid_macro.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.sql b/tests/queries/0_stateless/02711_server_uuid_macro.sql index f708156c0ae..f10ed7f8f6f 100644 --- a/tests/queries/0_stateless/02711_server_uuid_macro.sql +++ b/tests/queries/0_stateless/02711_server_uuid_macro.sql @@ -1,13 +1,13 @@ DROP TABLE IF EXISTS test; -- You can create a table with the {server_uuid} substituted. -CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; +CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'replica-{server_uuid}') ORDER BY x; -- The server UUID is correctly substituted. SELECT engine_full LIKE ('%replica-' || serverUUID()::String || '%') FROM system.tables WHERE database = currentDatabase() AND name = 'test'; -- An attempt to create a second table with the same UUID results in error. -CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } +CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } -- The macro {server_uuid} is special, not a configuration-type macro. It's normal that it is inaccessible with the getMacro function. SELECT getMacro('server_uuid'); -- { serverError NO_ELEMENTS_IN_CONFIG } From 6fe6e1fea5f31dc9f2684ae29df2e07a89d8c6f0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 19:55:45 +0000 Subject: [PATCH 28/52] Adopt tests to run in parallel or mark as no-parallel (for flaky check) --- .../02293_arrow_dictionary_indexes.sql | 2 +- .../02383_arrow_dict_special_cases.sh | 32 +++++++++---------- .../02455_one_row_from_csv_memory_usage.sh | 11 +++++-- 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql index ec68d1a4443..3ea229a1152 100644 --- a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql +++ b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql @@ -1,3 +1,3 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into function file(02293_data.arrow) select toLowCardinality(toString(number)) from numbers(300) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1; select * from file(02293_data.arrow) settings max_threads=1; diff --git a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh index bab0b57f116..40487f16551 100755 --- a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh +++ b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh @@ -6,24 +6,24 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +UNIQ_DEST_PATH=$USER_FILES_PATH/test-02383-$RANDOM-$RANDOM +mkdir -p $UNIQ_DEST_PATH -mkdir -p $USER_FILES_PATH/test_02383 -cp $CURDIR/data_arrow/dictionary*.arrow $USER_FILES_PATH/test_02383/ -cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/ -cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/ +cp $CURDIR/data_arrow/dictionary*.arrow $UNIQ_DEST_PATH/ +cp $CURDIR/data_arrow/corrupted.arrow $UNIQ_DEST_PATH/ +cp $CURDIR/data_arrow/dict_with_nulls.arrow $UNIQ_DEST_PATH/ -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary1.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary1.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary2.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary2.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary3.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary3.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/corrupted.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dict_with_nulls.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dict_with_nulls.arrow') settings max_threads=1" - -rm -rf $USER_FILES_PATH/test_02383 +rm -rf $UNIQ_DEST_PATH diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 9076d7a8a5f..0b6f91907d4 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -5,7 +5,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $user_files_path/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" +UNIQ_DEST_PATH=$user_files_path/test-02455-$RANDOM-$RANDOM +mkdir -p $UNIQ_DEST_PATH +cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $UNIQ_DEST_PATH/ + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" + +rm -rf $UNIQ_DEST_PATH From 211cea5e7c99119777d387a6b5331e4703e24510 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Apr 2023 22:50:21 +0200 Subject: [PATCH 29/52] Fix uncaught exception in case of parallel loader for hashed dictionaries Since ThreadPool::wait() rethrows the first exception (if any):
stacktrace 2023.04.09 12:53:33.629333 [ 22361 ] {} BaseDaemon: (version 22.13.1.1, build id: 5FB01DCAAFFF19F0A9A61E253567F90685989D2F) (from thread 23032) Terminate called for uncaught exception: 2023.04.09 12:53:33.630179 [ 23645 ] {} BaseDaemon: 2023.04.09 12:53:33.630213 [ 23645 ] {} BaseDaemon: Stack trace: 0x7f68b00baccc 0x7f68b006bef2 0x7f68b0056472 0x112a42fe 0x1c17f2a3 0x1c17f238 0xbf4bc3b 0x13961c6d 0x138ee529 0x138ed6bc 0x138dd2f0 0x138dd9c6 0x1571d0dd 0x16197c1f 0x161a231e 0x1619fc93 0x161a51b9 0x11151759 0x1115454e 0x7f68b00b8fd4 0x7f68b013966c 2023.04.09 12:53:33.630247 [ 23645 ] {} BaseDaemon: 3. ? @ 0x7f68b00baccc in ? 2023.04.09 12:53:33.630263 [ 23645 ] {} BaseDaemon: 4. gsignal @ 0x7f68b006bef2 in ? 2023.04.09 12:53:33.630273 [ 23645 ] {} BaseDaemon: 5. abort @ 0x7f68b0056472 in ? 2023.04.09 12:53:33.648815 [ 23645 ] {} BaseDaemon: 6. ./.build/./src/Daemon/BaseDaemon.cpp:456: terminate_handler() @ 0x112a42fe in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:33.651484 [ 23645 ] {} BaseDaemon: 7. ./.build/./contrib/llvm-project/libcxxabi/src/cxa_handlers.cpp:61: std::__terminate(void (*)()) @ 0x1c17f2a3 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:33.654080 [ 23645 ] {} BaseDaemon: 8. ./.build/./contrib/llvm-project/libcxxabi/src/cxa_handlers.cpp:79: std::terminate() @ 0x1c17f238 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:35.025565 [ 23645 ] {} BaseDaemon: 9. ? @ 0xbf4bc3b in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:36.495557 [ 23645 ] {} BaseDaemon: 10. DB::ParallelDictionaryLoader<(DB::DictionaryKeyType)0, true, true>::~ParallelDictionaryLoader() @ 0x13961c6d in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:37.833142 [ 23645 ] {} BaseDaemon: 11. DB::HashedDictionary<(DB::DictionaryKeyType)0, true, true>::loadData() @ 0x138ee529 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:39.124989 [ 23645 ] {} BaseDaemon: 12. DB::HashedDictionary<(DB::DictionaryKeyType)0, true, true>::HashedDictionary(DB::StorageID const&, DB::DictionaryStructure const&, std::__1::shared_ptr, DB::HashedDictionaryStorageConfiguration const&, std::__1::shared_ptr) @ 0x138ed6bc in /usr/lib/debug/usr/bin/clickhouse.debug
Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 0e5d18363e9..5cfac20e572 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -114,9 +114,18 @@ public: ~ParallelDictionaryLoader() { - for (auto & queue : shards_queues) - queue->clearAndFinish(); - pool.wait(); + try + { + for (auto & queue : shards_queues) + queue->clearAndFinish(); + + /// NOTE: It is OK to not pass the exception next, since on success finish() should be called which will call wait() + pool.wait(); + } + catch (...) + { + tryLogCurrentException(dictionary.log, "Exception had been thrown during parallel load of the dictionary"); + } } private: From 1187534545451970bd10eb10a986d5e33a8f0b01 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 21:26:39 +0000 Subject: [PATCH 30/52] Simpler way to resize pipeline --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 665630c3559..1b195d2eb47 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -797,7 +797,7 @@ Pipe StorageFile::read( /// It happens if globs in file(path, ...) expands to empty set i.e. no files to process if (num_streams > 0 && num_streams < max_num_streams) { - pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); + pipe.resize(max_num_streams); } return pipe; } From 98cabda66d32069bc1e143a9d7f85cf6369720c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 00:15:15 +0200 Subject: [PATCH 31/52] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 7c78edd52b4..5e05432420f 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 7c78edd52b4d65acc103c2f195818ffcabe6fe0d +Subproject commit 5e05432420f9692418e2e12aff09859e420b14a2 From c89eb29b0f203e0262891080464293bf153b7e1d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:24:42 -0300 Subject: [PATCH 32/52] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 7be933d67d7..c059b2c1e63 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, невозможно объединить их в "правом" порядке. Функция отслеживает `timestamp` наблюдаемых значений, поэтому возможно правильно упорядочить состояния во время слияния. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), хранящих данные, упорядоченные по некоторому временному бакету согласно timestamp (time-series), например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, их невозможно объединить их в правильном порядке без хранения исходного, неокругленного значения timestamp. Функция `deltaSumTimestamp` отслеживает исходные `timestamp` наблюдаемых значений, поэтому значения (состояния) функции правильно вычисляются во время слияния кусков. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вместо функции `deltaSumTimestamp`. From a4183204f9a6b38e40a8523e3c694e0fc0226756 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:27:13 -0300 Subject: [PATCH 33/52] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index c059b2c1e63..8b022241dac 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), хранящих данные, упорядоченные по некоторому временному бакету согласно timestamp (time-series), например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, их невозможно объединить их в правильном порядке без хранения исходного, неокругленного значения timestamp. Функция `deltaSumTimestamp` отслеживает исходные `timestamp` наблюдаемых значений, поэтому значения (состояния) функции правильно вычисляются во время слияния кусков. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), хранящих данные, упорядоченные по некоторому временному бакету согласно timestamp (time-series), например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порядке без хранения исходного, неокругленного значения timestamp. Функция `deltaSumTimestamp` отслеживает исходные `timestamp` наблюдаемых значений, поэтому значения (состояния) функции правильно вычисляются во время слияния кусков. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вместо функции `deltaSumTimestamp`. From 028a21658d98e6aad5051de2f2f9b1a94461fbfd Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:34:10 -0300 Subject: [PATCH 34/52] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index e08e69b7cf6..afcf2a48c23 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ title: deltaSumTimestamp Adds the difference between consecutive rows. If the difference is negative, it is ignored. -This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. +This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that store data ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the correct order, without storing the original, unrounded timestamp value. The `deltaSumTimestamp` function keeps track of the original `timestamp` of the values it's seen, so the values (states) of the function are correctly computed during merging of parts. To calculate the delta sum across an ordered collection you can simply use the [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) function. From 36871fad3498c807b63ab54c2bd9643bef3710b8 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:38:08 -0300 Subject: [PATCH 35/52] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 8b022241dac..50434419651 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), хранящих данные, упорядоченные по некоторому временному бакету согласно timestamp (time-series), например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порядке без хранения исходного, неокругленного значения timestamp. Функция `deltaSumTimestamp` отслеживает исходные `timestamp` наблюдаемых значений, поэтому значения (состояния) функции правильно вычисляются во время слияния кусков. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), хранящих данные, упорядоченные по некоторому округленному временному интервалу, согласно timestamp, например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порядке без хранения исходного, неокругленного значения timestamp. Функция `deltaSumTimestamp` отслеживает исходные `timestamp` наблюдаемых значений, поэтому значения (состояния) функции правильно вычисляются во время слияния кусков. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вместо функции `deltaSumTimestamp`. From c06a5ab85f9fa9af39d33c836f51cd8c353a83a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 01:04:23 +0200 Subject: [PATCH 36/52] Remove slow test from debug builds --- tests/queries/0_stateless/02703_jit_external_aggregation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02703_jit_external_aggregation.sh b/tests/queries/0_stateless/02703_jit_external_aggregation.sh index d1af5b8b8bc..2d1dda45de0 100755 --- a/tests/queries/0_stateless/02703_jit_external_aggregation.sh +++ b/tests/queries/0_stateless/02703_jit_external_aggregation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-asan, no-msan, no-tsan, no-ubsan +# Tags: long, no-asan, no-msan, no-tsan, no-ubsan, no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e1fa279c9875c35fe655adb460e4836dabf6a782 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 23:40:32 +0000 Subject: [PATCH 37/52] Fix tests for flaky check --- .../0_stateless/02211_jsonl_format_extension.sql | 2 +- .../02455_one_row_from_csv_memory_usage.sh | 13 ++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index a95b19f4e3b..61cc2a408fa 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10) SETTINGS engine_file_truncate_on_insert=1; select * from file('data.jsonl') order by x; diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 0b6f91907d4..1b0101e4f06 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -1,16 +1,15 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') -UNIQ_DEST_PATH=$user_files_path/test-02455-$RANDOM-$RANDOM -mkdir -p $UNIQ_DEST_PATH -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $UNIQ_DEST_PATH/ +cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" -rm -rf $UNIQ_DEST_PATH +rm $USER_FILES_PATH/10m_rows.csv.xz From 5cb00e13c3bff6721d1ea5dc26f5e39ba904f4a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 02:31:49 +0200 Subject: [PATCH 38/52] Add a test --- ...12_bool_better_exception_message.reference | 5 ++++ .../02712_bool_better_exception_message.sh | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02712_bool_better_exception_message.reference create mode 100755 tests/queries/0_stateless/02712_bool_better_exception_message.sh diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.reference b/tests/queries/0_stateless/02712_bool_better_exception_message.reference new file mode 100644 index 00000000000..aba60fe1061 --- /dev/null +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.reference @@ -0,0 +1,5 @@ +true +false +1 +1 +1 diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.sh b/tests/queries/0_stateless/02712_bool_better_exception_message.sh new file mode 100755 index 00000000000..ce6a4f4874b --- /dev/null +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +SCHEMADIR=$CURDIR/format_schemas + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'w' character" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": wtf}'); +END + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "expected 'false'" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": ftw}'); +END + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'{' character" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": {}}'); +END From 1f6b9809dd7bda0a0fa83be5f42e24f4b205128a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 02:32:04 +0200 Subject: [PATCH 39/52] Better exception messages for unparsed Bool --- src/DataTypes/Serializations/SerializationBool.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 81ad0ec46b1..41b5bf806e5 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -238,12 +238,15 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); bool value = false; - if (*istr.position() == 't' || *istr.position() == 'f') + char first_char = *istr.position(); + if (first_char == 't' || first_char == 'f') readBoolTextWord(value, istr); - else if (*istr.position() == '1' || *istr.position() == '0') + else if (first_char == '1' || first_char == '0') readBoolText(value, istr); else - throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Invalid boolean value, should be true/false, 1/0."); + throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, + "Invalid boolean value, should be true/false, 1/0, but it starts with the '{}' character.", first_char); + col->insert(value); } From a24cc9580cea69c9fc2e70db030fdfebd3d2a5c2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:11:10 +0200 Subject: [PATCH 40/52] Update syntax.md --- docs/en/sql-reference/syntax.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 63c5042f9e8..00d11ccff40 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -14,7 +14,7 @@ The `INSERT` query uses both parsers: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#syntax-expressions). +The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). Data can have any format. When a query is received, the server calculates no more than [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. It allows for avoiding issues with large `INSERT` queries. @@ -45,7 +45,7 @@ You can check whether a data type name is case-sensitive in the [system.data_typ In contrast to standard SQL, all other keywords (including functions names) are **case-sensitive**. -Keywords are not reserved; they are treated as such only in the corresponding context. If you use [identifiers](#syntax-identifiers) with the same name as the keywords, enclose them into double-quotes or backticks. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`. +Keywords are not reserved; they are treated as such only in the corresponding context. If you use [identifiers](#identifiers) with the same name as the keywords, enclose them into double-quotes or backticks. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`. ## Identifiers @@ -54,7 +54,7 @@ Identifiers are: - Cluster, database, table, partition, and column names. - Functions. - Data types. -- [Expression aliases](#syntax-expression_aliases). +- [Expression aliases](#expression_aliases). Identifiers can be quoted or non-quoted. The latter is preferred. @@ -149,7 +149,7 @@ For example, the following SQL defines parameters named `a`, `b`, `c` and `d` - SET param_a = 13; SET param_b = 'str'; SET param_c = '2022-08-04 18:30:53'; -SET param_d = {'10': [11, 12], '13': [14, 15]}'; +SET param_d = {'10': [11, 12], '13': [14, 15]}; SELECT {a: UInt32}, @@ -166,7 +166,7 @@ Result: If you are using `clickhouse-client`, the parameters are specified as `--param_name=value`. For example, the following parameter has the name `message` and it is retrieved as a `String`: -```sql +```bash clickhouse-client --param_message='hello' --query="SELECT {message: String}" ``` @@ -190,7 +190,7 @@ Query parameters are not general text substitutions which can be used in arbitra ## Functions Function calls are written like an identifier with a list of arguments (possibly empty) in round brackets. In contrast to standard SQL, the brackets are required, even for an empty argument list. Example: `now()`. -There are regular and aggregate functions (see the section “Aggregate functions”). Some aggregate functions can contain two lists of arguments in brackets. Example: `quantile (0.9) (x)`. These aggregate functions are called “parametric” functions, and the arguments in the first list are called “parameters”. The syntax of aggregate functions without parameters is the same as for regular functions. +There are regular and aggregate functions (see the section [Aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md)). Some aggregate functions can contain two lists of arguments in brackets. Example: `quantile (0.9) (x)`. These aggregate functions are called “parametric” functions, and the arguments in the first list are called “parameters”. The syntax of aggregate functions without parameters is the same as for regular functions. ## Operators @@ -199,7 +199,7 @@ For example, the expression `1 + 2 * 3 + 4` is transformed to `plus(plus(1, mult ## Data Types and Database Table Engines -Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an argument list in brackets. For more information, see the sections “Data types,” “Table engines,” and “CREATE”. +Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an argument list in brackets. For more information, see the sections [Data types](/docs/en/sql-reference/data-types/index.md), [Table engines](/docs/en/engines/table-engines/index.md), and [CREATE](/docs/en/sql-reference/statements/create/index.md). ## Expression Aliases @@ -211,17 +211,17 @@ expr AS alias - `AS` — The keyword for defining aliases. You can define the alias for a table name or a column name in a `SELECT` clause without using the `AS` keyword. - For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. + For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. - In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. + In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. - `expr` — Any expression supported by ClickHouse. - For example, `SELECT column_name * 2 AS double FROM some_table`. + For example, `SELECT column_name * 2 AS double FROM some_table`. -- `alias` — Name for `expr`. Aliases should comply with the [identifiers](#syntax-identifiers) syntax. +- `alias` — Name for `expr`. Aliases should comply with the [identifiers](#identifiers) syntax. - For example, `SELECT "table t".column_name FROM table_name AS "table t"`. + For example, `SELECT "table t".column_name FROM table_name AS "table t"`. ### Notes on Usage @@ -258,7 +258,7 @@ In this example, we declared table `t` with column `b`. Then, when selecting dat ## Asterisk -In a `SELECT` query, an asterisk can replace the expression. For more information, see the section “SELECT”. +In a `SELECT` query, an asterisk can replace the expression. For more information, see the section [SELECT](/docs/en/sql-reference/statements/select/index.md#asterisk). ## Expressions From 8eaf20e1294a67344260e5d967a4b02db9ef7dd7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:26:43 +0200 Subject: [PATCH 41/52] Update syntax.md --- docs/en/sql-reference/syntax.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 00d11ccff40..ea2df235c1a 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -14,7 +14,7 @@ The `INSERT` query uses both parsers: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). +The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). Data can have any format. When a query is received, the server calculates no more than [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. It allows for avoiding issues with large `INSERT` queries. @@ -108,7 +108,7 @@ Depending on the data format (input or output), `NULL` may have a different repr There are many nuances to processing `NULL`. For example, if at least one of the arguments of a comparison operation is `NULL`, the result of this operation is also `NULL`. The same is true for multiplication, addition, and other operations. For more information, read the documentation for each operation. -In queries, you can check `NULL` using the [IS NULL](../sql-reference/operators/index.md#operator-is-null) and [IS NOT NULL](../sql-reference/operators/index.md) operators and the related functions `isNull` and `isNotNull`. +In queries, you can check `NULL` using the [IS NULL](../sql-reference/operators/index.md#is-null) and [IS NOT NULL](../sql-reference/operators/index.md#is-not-null) operators and the related functions `isNull` and `isNotNull`. ### Heredoc @@ -213,7 +213,7 @@ expr AS alias For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. - In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. + In the [CAST](./functions/type-conversion-functions.md#castx-t) function, the `AS` keyword has another meaning. See the description of the function. - `expr` — Any expression supported by ClickHouse. @@ -254,7 +254,7 @@ Received exception from server (version 18.14.17): Code: 184. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: Aggregate function sum(b) is found inside another aggregate function in query. ``` -In this example, we declared table `t` with column `b`. Then, when selecting data, we defined the `sum(b) AS b` alias. As aliases are global, ClickHouse substituted the literal `b` in the expression `argMax(a, b)` with the expression `sum(b)`. This substitution caused the exception. You can change this default behavior by setting [prefer_column_name_to_alias](../operations/settings/settings.md#prefer_column_name_to_alias) to `1`. +In this example, we declared table `t` with column `b`. Then, when selecting data, we defined the `sum(b) AS b` alias. As aliases are global, ClickHouse substituted the literal `b` in the expression `argMax(a, b)` with the expression `sum(b)`. This substitution caused the exception. You can change this default behavior by setting [prefer_column_name_to_alias](../operations/settings/settings.md#prefer-column-name-to-alias) to `1`. ## Asterisk From 8e90b7d446521af0a40ec3c0096f62e7e9395321 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 03:31:43 +0200 Subject: [PATCH 42/52] Fixup --- .../queries/0_stateless/02712_bool_better_exception_message.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.sh b/tests/queries/0_stateless/02712_bool_better_exception_message.sh index ce6a4f4874b..1da70d7025b 100755 --- a/tests/queries/0_stateless/02712_bool_better_exception_message.sh +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SCHEMADIR=$CURDIR/format_schemas + $CLICKHOUSE_LOCAL < Date: Mon, 10 Apr 2023 03:39:13 +0200 Subject: [PATCH 43/52] Don't use type conversion with String query parameters --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index fad9d4bbfb2..71f28f9fb2c 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -102,7 +102,10 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; - ast = addTypeConversionToAST(std::make_shared(literal), type_name); + if (typeid_cast(&data_type)) + ast = std::make_shared(literal); + else + ast = addTypeConversionToAST(std::make_shared(literal), type_name); /// Keep the original alias. ast->setAlias(alias); From e71c7fe7449f18591a6a3bf375495fa792939ee8 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:48:58 +0200 Subject: [PATCH 44/52] Fix --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 71f28f9fb2c..7ed56dce9d0 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -102,7 +103,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; - if (typeid_cast(&data_type)) + if (typeid_cast(data_type.get())) ast = std::make_shared(literal); else ast = addTypeConversionToAST(std::make_shared(literal), type_name); From 345b33dc1fba2937fc758ddd70e0e5f7cd1af6d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 04:15:13 +0200 Subject: [PATCH 45/52] Slightly optimize Install --- programs/install/Install.cpp | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index b142159fbdf..d83e189f7ef 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -375,15 +375,22 @@ int mainEntryClickHouseInstall(int argc, char ** argv) try { - ReadBufferFromFile in(binary_self_path.string()); - WriteBufferFromFile out(main_bin_tmp_path.string()); - copyData(in, out); - out.sync(); + String source = binary_self_path.string(); + String destination = main_bin_tmp_path.string(); - if (0 != fchmod(out.getFD(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + /// Try to make a hard link first, as an optimization. + /// It is possible if the source and the destination are on the same filesystems. + if (0 != link(source.c_str(), destination.c_str())) + { + ReadBufferFromFile in(binary_self_path.string()); + WriteBufferFromFile out(main_bin_tmp_path.string()); + copyData(in, out); + out.sync(); + out.finalize(); + } + + if (0 != chmod(destination.c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR); - - out.finalize(); } catch (const Exception & e) { From 6738ab66600d9b6857af07b8d98bf2f0d5277348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 04:22:17 +0200 Subject: [PATCH 46/52] Fix TSan report in Kerberos --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 9453aec0d50..b56ce6ba690 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 9453aec0d50e5aff9b189051611b321b40935d02 +Subproject commit b56ce6ba690e1f320df1a64afa34980c3e462617 From 8524f6958a82e772919e02f2cb6647e8286d3a02 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 09:09:57 +0200 Subject: [PATCH 47/52] Update src/Interpreters/ReplaceQueryParameterVisitor.cpp Co-authored-by: Alexey Milovidov --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 7ed56dce9d0..c92161c72c3 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -103,6 +103,9 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; + /// If it's a String, substitute it in the form of a string literal without CAST + /// to enable substitutions in simple queries that don't support expressions + /// (such as CREATE USER). if (typeid_cast(data_type.get())) ast = std::make_shared(literal); else From d564fe4e6446f503250d02f806c1f3eab30cad9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Apr 2023 10:54:37 +0000 Subject: [PATCH 48/52] Fixing build --- src/Storages/System/StorageSystemZooKeeper.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index c96c8ec60ae..e21bd887ecb 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -356,7 +356,7 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!isPathNode(node.children.at(0))) return; - auto value = node.children.at(1); + const auto * value = node.children.at(1); if (!value->column) return; @@ -504,7 +504,7 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const } ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, ContextPtr context_) - : SourceStepWithFilter({.header = std::move(header)}) + : SourceStepWithFilter({.header = header}) , storage_limits(query_info.storage_limits) , context(std::move(context_)) { From 450e8207285b411f614564011a2845b9e22aaa6b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 16:18:48 +0200 Subject: [PATCH 49/52] Docs: Add SETTINGS to executable table function --- docs/en/sql-reference/table-functions/executable.md | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index 22c74eb8cfa..5a24c3ab11d 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -20,7 +20,7 @@ A key advantage between ordinary UDF functions and the `executable` table functi The `executable` table function requires three parameters and accepts an optional list of input queries: ```sql -executable(script_name, format, structure, [input_query...]) +executable(script_name, format, structure, [input_query...] [,SETTINGS ...]) ``` - `script_name`: the file name of the script. saved in the `user_scripts` folder (the default folder of the `user_scripts_path` setting) @@ -83,6 +83,15 @@ The response looks like: └────┴────────────┘ ``` +## Settings + +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Default value is `false`. +- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. Default value is `16`. +- `max_command_execution_time` — Maximum executable script command execution time for processing block of data. Specified in seconds. Default value is 10. +- `command_termination_timeout` — executable script should contain main read-write loop. After table function is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. +- `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. + ## Passing Query Results to a Script Be sure to check out the example in the `Executable` table engine on [how to pass query results to a script](../../engines/table-engines/special/executable.md#passing-query-results-to-a-script). Here is how you execute the same script in that example using the `executable` table function: @@ -94,4 +103,4 @@ SELECT * FROM executable( 'id UInt64, sentiment Float32', (SELECT id, comment FROM hackernews WHERE id > 0 AND comment != '' LIMIT 20) ); -``` \ No newline at end of file +``` From 79b83c4fd2249daaaa606ab907bd72dc0a2afde8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 8 Apr 2023 06:47:21 +0200 Subject: [PATCH 50/52] Remove superfluous includes of logger_userful.h from headers Signed-off-by: Azat Khuzhin --- .../AggregateFunctionMaxIntersections.h | 1 - .../AggregateFunctionSparkbar.h | 1 - .../AggregateFunctionSumMap.h | 1 - src/BridgeHelper/IBridgeHelper.h | 1 - src/BridgeHelper/XDBCBridgeHelper.h | 1 - src/Client/Connection.cpp | 3 ++- src/Client/Connection.h | 1 - src/Client/LocalConnection.cpp | 1 + src/Client/LocalConnection.h | 1 + src/Common/AsynchronousMetrics.cpp | 1 + src/Common/CacheBase.h | 1 - src/Common/Config/ConfigProcessor.cpp | 1 + src/Common/Config/ConfigProcessor.h | 3 ++- src/Common/Config/YAMLParser.h | 8 +++---- src/Common/Config/configReadClient.cpp | 1 + src/Common/DNSResolver.cpp | 1 + src/Common/DNSResolver.h | 3 ++- src/Common/ErrorHandlers.h | 1 - src/Common/FileChecker.cpp | 6 ++++- src/Common/FileChecker.h | 6 +++-- src/Common/LRUCachePolicy.h | 6 ++--- src/Common/OvercommitTracker.h | 2 +- src/Common/PoolBase.h | 11 ++++++--- src/Common/SLRUCachePolicy.h | 6 ++--- src/Common/ThreadProfileEvents.cpp | 1 + src/Common/ThreadProfileEvents.h | 4 +++- src/Common/ThreadStatus.cpp | 2 ++ src/Common/ZooKeeper/Common.h | 2 -- src/Common/ZooKeeper/ZooKeeper.cpp | 1 + src/Common/ZooKeeper/ZooKeeper.h | 1 - src/Common/ZooKeeper/ZooKeeperLock.cpp | 2 ++ src/Common/ZooKeeper/ZooKeeperLock.h | 3 ++- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 1 + src/Common/scope_guard_safe.h | 1 - src/Coordination/KeeperDispatcher.cpp | 1 + src/Coordination/KeeperDispatcher.h | 1 - src/Coordination/KeeperLogStore.cpp | 1 + src/Coordination/KeeperLogStore.h | 1 - src/Coordination/KeeperSnapshotManagerS3.h | 1 - src/Coordination/KeeperStateMachine.cpp | 3 ++- src/Coordination/KeeperStateMachine.h | 1 - src/Coordination/KeeperStateManager.cpp | 1 + src/Coordination/ZooKeeperDataReader.cpp | 1 + src/Coordination/ZooKeeperDataReader.h | 1 - src/Core/PostgreSQL/PoolWithFailover.cpp | 1 + src/Core/PostgreSQL/PoolWithFailover.h | 1 - src/Core/SortDescription.cpp | 1 + src/Daemon/BaseDaemon.h | 1 - src/Dictionaries/CacheDictionary.h | 1 - src/Dictionaries/ExecutableDictionarySource.h | 1 - .../ExecutablePoolDictionarySource.h | 1 - src/Dictionaries/IPAddressDictionary.cpp | 1 + src/Dictionaries/IPAddressDictionary.h | 1 - .../PostgreSQLDictionarySource.cpp | 1 + src/Dictionaries/PostgreSQLDictionarySource.h | 1 - src/Disks/DiskLocal.h | 1 - .../IO/CachedOnDiskReadBufferFromFile.cpp | 3 ++- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 - src/Disks/StoragePolicy.cpp | 1 + src/Disks/StoragePolicy.h | 1 - src/Functions/FunctionsDecimalArithmetics.h | 1 - src/Functions/array/arrayEnumerateExtended.h | 1 + src/Functions/array/arrayEnumerateRanked.h | 1 + src/Functions/geometryConverters.h | 1 - src/IO/S3/AWSLogger.cpp | 3 +++ src/IO/S3/AWSLogger.h | 4 +++- src/IO/S3/Client.h | 1 - src/IO/S3/Credentials.h | 1 - src/IO/WriteBufferFromS3.h | 1 - src/Interpreters/Aggregator.cpp | 23 ++++++++++--------- src/Interpreters/Aggregator.h | 1 - src/Interpreters/Cache/FileCache.h | 1 - .../Cache/LRUFileCachePriority.cpp | 1 + src/Interpreters/Cache/LRUFileCachePriority.h | 1 - src/Interpreters/DNSCacheUpdater.cpp | 1 + src/Interpreters/DirectJoin.cpp | 4 ++-- src/Interpreters/DirectJoin.h | 1 - src/Interpreters/ExternalLoader.cpp | 3 ++- src/Interpreters/ExternalLoader.h | 3 ++- src/Interpreters/FilesystemCacheLog.h | 1 - src/Interpreters/FullSortingMergeJoin.h | 1 + src/Interpreters/HashJoin.h | 1 - src/Interpreters/IJoin.h | 1 - src/Interpreters/InterserverCredentials.h | 1 - src/Interpreters/MergeJoin.cpp | 1 + src/Interpreters/Set.h | 1 - src/Interpreters/TableJoin.h | 1 - src/Interpreters/ThreadStatusExt.cpp | 1 + src/Planner/Planner.cpp | 1 + .../Executors/CompletedPipelineExecutor.cpp | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 1 + src/Processors/Executors/PipelineExecutor.h | 1 - .../PushingAsyncPipelineExecutor.cpp | 3 ++- src/Processors/PingPongProcessor.h | 1 - .../CreateSetAndFilterOnTheFlyStep.cpp | 1 + src/Processors/Sources/ShellCommandSource.h | 1 - .../AggregatingInOrderTransform.cpp | 1 + .../Transforms/AggregatingTransform.cpp | 1 + .../Transforms/AggregatingTransform.h | 1 + .../Transforms/CreatingSetsTransform.cpp | 1 + .../Transforms/MemoryBoundMerging.h | 1 - .../Transforms/MergeJoinTransform.cpp | 10 ++++++++ .../Transforms/MergeJoinTransform.h | 10 +------- .../Transforms/MergeSortingTransform.cpp | 1 + .../Transforms/MergeSortingTransform.h | 1 - .../Transforms/MergingAggregatedTransform.cpp | 1 + src/Server/HTTPHandlerFactory.h | 1 - src/Server/PostgreSQLHandler.h | 1 - src/Server/ProxyV1Handler.cpp | 1 + src/Server/ProxyV1Handler.h | 1 - src/Storages/Cache/RemoteCacheController.h | 1 - .../FileLog/FileLogDirectoryWatcher.cpp | 1 + .../FileLog/FileLogDirectoryWatcher.h | 1 - src/Storages/HDFS/StorageHDFS.h | 1 - src/Storages/Hive/StorageHive.h | 1 - src/Storages/KVStorageUtils.h | 1 - src/Storages/MergeTree/AsyncBlockIDsCache.h | 1 - src/Storages/MergeTree/DataPartsExchange.cpp | 1 + src/Storages/MergeTree/DataPartsExchange.h | 4 ++-- src/Storages/MergeTree/IDataPartStorage.h | 1 - .../MergeTree/IMergedBlockOutputStream.cpp | 1 + .../MergeTree/MergeTreeBackgroundExecutor.cpp | 3 ++- .../MergeTree/MergeTreeBackgroundExecutor.h | 3 +-- .../MergeTreeBaseSelectProcessor.cpp | 1 + .../MergeTreeInOrderSelectProcessor.h | 1 + .../MergeTree/MergeTreePartsMover.cpp | 1 + .../MergeTree/MergeTreePrefetchedReadPool.cpp | 1 + .../MergeTree/MergeTreeRangeReader.cpp | 1 + src/Storages/MergeTree/MergeTreeRangeReader.h | 1 - src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 ++- .../MergeTreeReverseSelectProcessor.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 3 ++- .../MergeTree/MergedBlockOutputStream.cpp | 1 + .../PartMovesBetweenShardsOrchestrator.h | 1 - src/Storages/MergeTree/PartitionPruner.cpp | 1 + .../MergeTree/ReplicatedMergeMutateTaskBase.h | 1 - .../ReplicatedMergeTreeAttachThread.h | 1 - .../ReplicatedMergeTreeCleanupThread.h | 1 - .../ReplicatedMergeTreePartCheckThread.h | 1 - .../ReplicatedMergeTreeRestartingThread.h | 1 - .../MaterializedPostgreSQLConsumer.h | 1 - src/Storages/StorageDistributed.h | 1 - src/Storages/StorageExecutable.h | 1 - src/Storages/StorageFile.h | 1 - src/Storages/StorageKeeperMap.h | 1 + src/Storages/StorageS3.h | 1 - src/Storages/StorageSet.cpp | 1 + src/Storages/System/StorageSystemReplicas.cpp | 1 + 150 files changed, 142 insertions(+), 130 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 2c54293eeec..5074e491f60 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index 78f7e9fcefa..30e107bc4db 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index f51ec423c69..b30f5ff5220 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -18,7 +18,6 @@ #include #include #include -#include #include diff --git a/src/BridgeHelper/IBridgeHelper.h b/src/BridgeHelper/IBridgeHelper.h index d4762087cc1..272d97c8a78 100644 --- a/src/BridgeHelper/IBridgeHelper.h +++ b/src/BridgeHelper/IBridgeHelper.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 00a661a1fc4..44104f26f63 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6643a94c3bc..11bba4f1448 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -22,7 +22,8 @@ #include #include #include -#include "Core/Block.h" +#include +#include #include #include #include diff --git a/src/Client/Connection.h b/src/Client/Connection.h index b86567e2ed0..5f79b365199 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -1,6 +1,5 @@ #pragma once -#include #include diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 712ff5f5a31..4fb80f0ea04 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 3e6fc007fb9..fb8f9003364 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 99073d79bcd..32d54b7644e 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 4ae313d7ecf..b176170cc1f 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -12,7 +12,6 @@ #include #include -#include #include diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index b632ea95928..5bbc8eae0de 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index aa8ac71446f..0ca3e46db88 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -16,9 +16,10 @@ #include #include #include -#include +namespace Poco { class Logger; } + namespace zkutil { class ZooKeeperNodeCache; diff --git a/src/Common/Config/YAMLParser.h b/src/Common/Config/YAMLParser.h index b986fc2d895..a00972b813c 100644 --- a/src/Common/Config/YAMLParser.h +++ b/src/Common/Config/YAMLParser.h @@ -2,11 +2,11 @@ #include "config.h" -#include - +#include +#include +#include #include -#include "Poco/DOM/AutoPtr.h" -#include +#include #if USE_YAML_CPP diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index e5308bc3bc7..44d338c07af 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -4,6 +4,7 @@ #include "ConfigProcessor.h" #include #include +#include namespace fs = std::filesystem; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 81e2624d6db..b6a68bdfb45 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index a05456d3de8..84715b392a8 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -5,9 +5,10 @@ #include #include #include -#include +namespace Poco { class Logger; } + namespace DB { diff --git a/src/Common/ErrorHandlers.h b/src/Common/ErrorHandlers.h index f55b6c83a69..301377bff83 100644 --- a/src/Common/ErrorHandlers.h +++ b/src/Common/ErrorHandlers.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index aa6b9c90a4c..a6e37654ff1 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include #include @@ -25,7 +27,9 @@ FileChecker::FileChecker(const String & file_info_path_) : FileChecker(nullptr, { } -FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_)) +FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) + : disk(std::move(disk_)) + , log(&Poco::Logger::get("FileChecker")) { setPath(file_info_path_); try diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 1beab31ec8f..bb0383e4b56 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -1,8 +1,10 @@ #pragma once -#include #include +#include +#include +namespace Poco { class Logger; } namespace DB { @@ -46,7 +48,7 @@ private: size_t getRealFileSize(const String & path_) const; const DiskPtr disk; - const Poco::Logger * log = &Poco::Logger::get("FileChecker"); + const Poco::Logger * log; String files_info_path; std::map map; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 4aee2135af7..49c2fc4541b 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { /// Cache policy LRU evicts entries which are not used for a long time. @@ -174,7 +172,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } @@ -192,7 +190,7 @@ private: if (current_size_in_bytes > (1ull << 63)) { - LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 598b877ef3c..f40a70fe7cd 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include #include #include diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 96a18ee6591..a9c595c440c 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -144,12 +144,17 @@ public: return Entry(*items.back()); } - LOG_INFO(log, "No free connections in pool. Waiting."); - if (timeout < 0) + { + LOG_INFO(log, "No free connections in pool. Waiting undefinitelly."); available.wait(lock); + } else - available.wait_for(lock, std::chrono::microseconds(timeout)); + { + auto timeout_ms = std::chrono::microseconds(timeout); + LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); + available.wait_for(lock, timeout_ms); + } } } diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index e36bca83c61..810c8c335a0 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { @@ -236,7 +234,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } @@ -264,7 +262,7 @@ private: if (current_size_in_bytes > (1ull << 63)) { - LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 76a4d8b1adf..a94fd81559a 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -23,6 +23,7 @@ #include #include +#include namespace ProfileEvents diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index 5a03a9e8555..c4703a69998 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -2,11 +2,13 @@ #include #include +#include +#include #include #include #include #include -#include +#include #if defined(OS_LINUX) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1b783aa9ec4..ac6cfdc2cd7 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include diff --git a/src/Common/ZooKeeper/Common.h b/src/Common/ZooKeeper/Common.h index a2956706c8f..1a1328588e3 100644 --- a/src/Common/ZooKeeper/Common.h +++ b/src/Common/ZooKeeper/Common.h @@ -3,12 +3,10 @@ #include #include -#include namespace zkutil { using GetZooKeeper = std::function; -using GetZooKeeperWithFaultInjection = std::function; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 78f9e3da83b..a8da0dff0cc 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -15,6 +15,7 @@ #include "Common/ZooKeeper/IKeeper.h" #include #include +#include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index fc4fe95d9b6..8e7639b8cc1 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeperLock.cpp b/src/Common/ZooKeeper/ZooKeeperLock.cpp index a52c942a35f..6ee1c380efb 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.cpp +++ b/src/Common/ZooKeeper/ZooKeeperLock.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/ZooKeeperLock.h b/src/Common/ZooKeeper/ZooKeeperLock.h index 755ca1333b8..146527c6c94 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.h +++ b/src/Common/ZooKeeper/ZooKeeperLock.h @@ -3,7 +3,8 @@ #include #include #include -#include + +namespace Poco { class Logger; } namespace zkutil { diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index a39a083cf33..130590ceb40 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Common/scope_guard_safe.h b/src/Common/scope_guard_safe.h index f098fd95f00..2befb58870a 100644 --- a/src/Common/scope_guard_safe.h +++ b/src/Common/scope_guard_safe.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include /// Same as SCOPE_EXIT() but block the MEMORY_LIMIT_EXCEEDED errors. diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 1828182751d..2cde7c2465e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 77b5510cbb3..4b8b134cf8f 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index d1bd2f9db18..10619a44517 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 108241e024e..e48e3a32463 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 197f528b192..eff7868bba9 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -9,7 +9,6 @@ #if USE_AWS_S3 #include #include -#include #include #endif diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 632aaec6b54..7c55739a96f 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -8,9 +8,10 @@ #include #include #include -#include "Common/ZooKeeper/ZooKeeperCommon.h" +#include #include #include +#include #include "Coordination/KeeperStorage.h" diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index e4f0295db99..5af5bc05b0f 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -8,7 +8,6 @@ #include #include -#include namespace DB diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cfb3519e597..70687ba471c 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 5fa67a60b4b..2d4f64e033f 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include diff --git a/src/Coordination/ZooKeeperDataReader.h b/src/Coordination/ZooKeeperDataReader.h index 6da6fd498af..8fd86ba99e2 100644 --- a/src/Coordination/ZooKeeperDataReader.h +++ b/src/Coordination/ZooKeeperDataReader.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include namespace DB { diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 22cd88c0764..3655681c515 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index afef2933d29..bf9c34e6723 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -8,7 +8,6 @@ #include "ConnectionHolder.h" #include #include -#include #include #include diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 66ca1539b71..ae6aedf202d 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #if USE_EMBEDDED_COMPILER #include diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index f90f403364b..7aa1e8ad1a0 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index e19c4a66b1f..5203415005f 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -9,7 +9,6 @@ #include -#include #include #include diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 0456d3cafef..c7067a62893 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 1fc10d18b76..e8cc6e83406 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 803f607a3a7..ff1c784750b 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index e1fabb89a7e..67827c6524e 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -11,7 +11,6 @@ #include #include #include -#include #include "DictionaryStructure.h" #include "IDictionary.h" #include "IDictionarySource.h" diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 9f254da0b11..8ec78308392 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #endif diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 8ecf56a9430..1305333458b 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -8,7 +8,6 @@ #include "ExternalQueryBuilder.h" #include #include -#include #include diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 14e29904422..d6182463ebf 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5eaee2e3026..f48935da7bf 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -4,8 +4,9 @@ #include #include #include -#include #include +#include +#include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 14e8ea6c7e7..d3c265a522b 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index bcdc97983be..b49f4dafef0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -9,7 +9,6 @@ #include #include #include -#include namespace DB diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index ec0f201b801..f4be8b8fe86 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 7e6aff7bbda..69cfb830818 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -10,7 +10,6 @@ #include #include #include -#include #include #include diff --git a/src/Functions/FunctionsDecimalArithmetics.h b/src/Functions/FunctionsDecimalArithmetics.h index aa4afc68707..79e10d215a9 100644 --- a/src/Functions/FunctionsDecimalArithmetics.h +++ b/src/Functions/FunctionsDecimalArithmetics.h @@ -10,7 +10,6 @@ #include #include -#include #include #include diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index 3f145c05b54..cf38afcfa5a 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 8a348c07421..0733f1e2d43 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 85e887dd498..97162fa9dd0 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -18,7 +18,6 @@ #include #include -#include namespace DB { diff --git a/src/IO/S3/AWSLogger.cpp b/src/IO/S3/AWSLogger.cpp index 48c30ccf881..d6162823aee 100644 --- a/src/IO/S3/AWSLogger.cpp +++ b/src/IO/S3/AWSLogger.cpp @@ -2,7 +2,10 @@ #if USE_AWS_S3 +#include +#include #include +#include namespace { diff --git a/src/IO/S3/AWSLogger.h b/src/IO/S3/AWSLogger.h index 897c0e8d964..fdb6eed1f86 100644 --- a/src/IO/S3/AWSLogger.h +++ b/src/IO/S3/AWSLogger.h @@ -4,8 +4,10 @@ #if USE_AWS_S3 #include +#include +#include -#include +namespace Poco { class Logger; } namespace DB::S3 { diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 5c68fca6f04..369558ae105 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -4,7 +4,6 @@ #if USE_AWS_S3 -#include #include #include diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index cd9072f9765..22cca76109f 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -9,7 +9,6 @@ # include # include -# include # include diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 7c3c439f9c3..5fa934b886e 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -9,7 +9,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d6fbf072d05..03227d8663a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -8,9 +8,6 @@ #endif #include -#include -#include -#include #include #include #include @@ -21,14 +18,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -37,6 +26,18 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index e2f76f31466..5fb94c5f4e8 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -5,7 +5,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index afafa39c4c6..85c91e8033e 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -12,7 +12,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index c20379e1fc1..4987d143f52 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace CurrentMetrics { diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 2345d3c47db..2a3c4f3c801 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index bf88d19b7ef..c6a38d85acb 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index e148db1d8e6..cfefd7c5a91 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -1,7 +1,7 @@ #include - -#include #include +#include +#include namespace DB { diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index bdbd155dc36..644b66a9d99 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -1,6 +1,5 @@ #pragma once -#include #include diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 04a116ec0c7..05418003dba 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -9,8 +9,9 @@ #include #include #include -#include #include +#include +#include #include #include #include diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 3ce7c40ef03..49b4ea77e0d 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -6,10 +6,11 @@ #include #include #include -#include #include #include +#include +namespace Poco { class Logger; } namespace DB { diff --git a/src/Interpreters/FilesystemCacheLog.h b/src/Interpreters/FilesystemCacheLog.h index bf5361ef324..1b22d561c51 100644 --- a/src/Interpreters/FilesystemCacheLog.h +++ b/src/Interpreters/FilesystemCacheLog.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a94d7a7dfc6..7318d1d24a1 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index b29b6e617c8..79fac60d8bc 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 69d69ce30a6..83067b0eab7 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/InterserverCredentials.h b/src/Interpreters/InterserverCredentials.h index c4370c83e9b..33176c943a4 100644 --- a/src/Interpreters/InterserverCredentials.h +++ b/src/Interpreters/InterserverCredentials.h @@ -2,7 +2,6 @@ #include #include -#include #include namespace DB diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 07f5ae31ed6..d31510c2fb5 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 00eff614c7c..e593f71cfbe 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -8,7 +8,6 @@ #include #include -#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 95471885a2a..99b683b7713 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -19,7 +19,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 72ce08bf653..2e051030e38 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #if defined(OS_LINUX) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a8e0d80ce8c..08efa6653a7 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1,6 +1,7 @@ #include #include +#include #include diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index e624ecd52de..acbb4098145 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -5,8 +5,8 @@ #include #include #include -#include #include +#include namespace DB { diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 994051fd697..441a66b16d7 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 147e5b2744a..e6d33e2849b 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 3aec7608e6d..e026dbc50d6 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -5,8 +5,9 @@ #include #include #include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 0f2fffbd70f..976b29f0d8a 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 07137e87736..fe362f64b96 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 80ba1d59adf..b9afaa345cf 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -2,7 +2,6 @@ #include -#include #include #include diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 95e9b953840..3d5a910f054 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 69eeb76bf53..4bd000797a6 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 048b69adae6..beb23a91c88 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace CurrentMetrics { diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6a8d08cb661..f65e72c2723 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -6,6 +6,7 @@ #include #include +#include #include diff --git a/src/Processors/Transforms/MemoryBoundMerging.h b/src/Processors/Transforms/MemoryBoundMerging.h index 3193a07a0bd..607087fb39c 100644 --- a/src/Processors/Transforms/MemoryBoundMerging.h +++ b/src/Processors/Transforms/MemoryBoundMerging.h @@ -6,7 +6,6 @@ #include #include -#include namespace DB { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index abeef0f9a47..46abd5c891d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -302,6 +302,16 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( } } +void MergeJoinAlgorithm::logElapsed(double seconds) +{ + LOG_TRACE(log, + "Finished pocessing in {} seconds" + ", left: {} blocks, {} rows; right: {} blocks, {} rows" + ", max blocks loaded to memory: {}", + seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], + stat.max_blocks_loaded); +} + static void prepareChunk(Chunk & chunk) { if (!chunk) diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 58ac652cb18..6bf4484df24 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -233,15 +233,7 @@ public: virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; - void logElapsed(double seconds) - { - LOG_TRACE(log, - "Finished pocessing in {} seconds" - ", left: {} blocks, {} rows; right: {} blocks, {} rows" - ", max blocks loaded to memory: {}", - seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], - stat.max_blocks_loaded); - } + void logElapsed(double seconds); private: std::optional handleAnyJoinState(); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index eebdd678a4b..ecf14a81c00 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index c64c93393ce..e8c180b6903 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 9d0be86ff83..e4955d06859 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ebdfa954bf7..76460dcc4a8 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -6,7 +6,6 @@ #include #include #include -#include #include diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index 6fc128e3883..f20af3df02c 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -4,7 +4,6 @@ #include "config.h" #include #include -#include #include "IServer.h" #if USE_SSL diff --git a/src/Server/ProxyV1Handler.cpp b/src/Server/ProxyV1Handler.cpp index cd5fe29112a..56621940a23 100644 --- a/src/Server/ProxyV1Handler.cpp +++ b/src/Server/ProxyV1Handler.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include diff --git a/src/Server/ProxyV1Handler.h b/src/Server/ProxyV1Handler.h index e56f4cd3545..b50c2acbc55 100644 --- a/src/Server/ProxyV1Handler.h +++ b/src/Server/ProxyV1Handler.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 18732acc273..fafe363bbd4 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 3c5342a1e83..9d488616e85 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4368f284c9d..9b7afcf8e12 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -2,7 +2,6 @@ #include -#include #include #include diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 585f5df6ceb..b123834e981 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 363042621c7..b4afb2421b1 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -7,7 +7,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index 0574539f4c7..c3bb2aefa62 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index a661d00f8a6..91d549a0501 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index c6efe9c9589..a80870448f1 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 20c15039a2d..57fd0d5cff0 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -1,13 +1,12 @@ #pragma once -#include "Storages/MergeTree/MergeTreePartInfo.h" +#include #include #include #include #include #include #include -#include #include @@ -21,6 +20,7 @@ namespace DB { class StorageReplicatedMergeTree; +class PooledReadWriteBufferFromHTTP; namespace DataPartsExchange { diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 4d7212eb17b..2d61f9ee77b 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -11,7 +11,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 37da6014d1b..21bead2864a 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index f2efbb2ba8d..6f1e41d2791 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -1,12 +1,13 @@ #include +#include #include #include #include #include -#include #include +#include namespace CurrentMetrics diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 1ed4fc2dabc..9bfea32c7f9 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -9,12 +9,11 @@ #include #include - #include #include +#include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index a41b697f9e6..24980cdb82c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h index f7c3f294658..da332c64232 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 6f83a82e4e6..e1da57744b3 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 114039f7ff7..e184f84528b 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 5cba3e34169..3f8a90ca175 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 5ffd464cfe2..f180787c869 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -1,6 +1,5 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 54a040724fc..29be06b4e6a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,8 +1,9 @@ #include #include #include -#include "Common/Stopwatch.h" +#include #include +#include #include diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index fd25748050a..618f2855695 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 4539e0b36c5..ef6013f5c65 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index fabf2acdad3..93e07a38bb1 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -4,15 +4,16 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include #include -#include "Storages/MergeTree/DataPartStorageOnDiskFull.h" #include namespace DB diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ced43ae25b0..715c218c2db 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index 24454b897af..af21022953c 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 61293888f10..35b2d5db3b5 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index fb8f01437ef..a7bf1290274 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h index a491a06d6a5..222b30b519b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 35838625bbe..76b9ee4a575 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 170b2ca1f60..e7b0e224d9b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index b5314de9dcc..8f5721e440d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 3412e6e422f..02cbedb4dd5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 3a7fae44708..417fb6447bf 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 2393920fa3c..37455385675 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 03b3aacb67f..8d622fdf3a2 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -3,7 +3,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index f71ff3cc65a..552e6b35fe8 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f4d915e9c55..982d4e4681e 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 7c5ba497ec9..e11b71d8777 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 240d452fe29..8832d9d4983 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include From 8b0adf58e52d78c2fe0a734e383bd0b4e4d22c34 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 10 Apr 2023 10:17:43 -0600 Subject: [PATCH 51/52] Update formats.md Including an Avro example with DESCRIBE url in S3 --- docs/en/interfaces/formats.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index de8357345db..157a852332f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1877,6 +1877,13 @@ Column names must: Output Avro file compression and sync interval can be configured with [output_format_avro_codec](/docs/en/operations/settings/settings-formats.md/#output_format_avro_codec) and [output_format_avro_sync_interval](/docs/en/operations/settings/settings-formats.md/#output_format_avro_sync_interval) respectively. +### Example Data {#example-data-avro} + +Using the ClickHouse [DESCRIBE](/docs/en/sql-reference/statements/describe-table) function, you can quickly view the inferred format of an Avro file like the following example. This example includes the URL of a publicly accessible Avro file in the ClickHouse S3 public bucket: + +``` DESCRIBE url('https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/hits.avro','Avro'); +``` + ## AvroConfluent {#data-format-avro-confluent} AvroConfluent supports decoding single-object Avro messages commonly used with [Kafka](https://kafka.apache.org/) and [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html). From 23a087945209ecb92f74f1fdc4c7c952f13841e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 21:17:08 +0200 Subject: [PATCH 52/52] Remove strange code --- src/Storages/StorageFile.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 03b3aacb67f..27cf40a0bf1 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -14,8 +14,6 @@ namespace DB class StorageFile final : public IStorage { -friend class partitionedstoragefilesink; - public: struct CommonArguments : public WithContext {