From dd8680018ff6df4e940b84b0e2a21f05ef0c1756 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 16:43:20 +0100 Subject: [PATCH 01/66] Analyzer: WIP on test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed --- src/Interpreters/getHeaderForProcessingStage.cpp | 2 +- src/Storages/StorageDistributed.cpp | 1 - src/Storages/StorageSnapshot.cpp | 8 ++++++++ src/Storages/StorageSnapshot.h | 3 +++ tests/analyzer_integration_broken_tests.txt | 1 - 5 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index d16e01ef2d2..1daf776b8bc 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -153,7 +153,7 @@ Block getHeaderForProcessingStage( if (context->getSettingsRef().allow_experimental_analyzer) { - auto storage = std::make_shared(storage_snapshot->storage.getStorageID(), storage_snapshot->metadata->getColumns()); + auto storage = std::make_shared(storage_snapshot->storage.getStorageID(), storage_snapshot->getAllColumnsDescription()); InterpreterSelectQueryAnalyzer interpreter(query, context, storage, SelectQueryOptions(processed_stage).analyze()); result = interpreter.getSampleBlock(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5fb404da1cf..6922261a823 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -295,7 +295,6 @@ NamesAndTypesList StorageDistributed::getVirtuals() const /// NOTE This is weird. Most of these virtual columns are part of MergeTree /// tables info. But Distributed is general-purpose engine. return NamesAndTypesList{ - NameAndTypePair("_table", std::make_shared(std::make_shared())), NameAndTypePair("_part", std::make_shared(std::make_shared())), NameAndTypePair("_part_index", std::make_shared()), NameAndTypePair("_part_uuid", std::make_shared()), diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 34c092c7208..8f5f4209efc 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -39,6 +39,14 @@ void StorageSnapshot::init() system_columns[BlockNumberColumn::name] = BlockNumberColumn::type; } +ColumnsDescription StorageSnapshot::getAllColumnsDescription() const +{ + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); + auto column_names_and_types = getColumns(get_column_options); + + return ColumnsDescription{column_names_and_types}; +} + NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const { auto all_columns = getMetadataForQuery()->getColumns().get(options); diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index d62e118e1f2..a5724f04967 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -62,6 +62,9 @@ struct StorageSnapshot std::shared_ptr clone(DataPtr data_) const; + /// Get columns description + ColumnsDescription getAllColumnsDescription() const; + /// Get all available columns with types according to options. NamesAndTypesList getColumns(const GetColumnsOptions & options) const; diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index c04ed440c18..31f626a23f7 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -6,7 +6,6 @@ test_distributed_type_object/test.py::test_distributed_type_object test_executable_table_function/test.py::test_executable_function_input_python test_mask_sensitive_info/test.py::test_encryption_functions test_merge_table_over_distributed/test.py::test_global_in -test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster From 093f629e4ed7b650efe6571bb7c7c36112b0d9f5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 14 Feb 2024 14:12:44 +0000 Subject: [PATCH 02/66] Update reference file --- .../0_stateless/02890_describe_table_options.reference | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 5d99df36bb4..8842eca3311 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -65,7 +65,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom │ t │ Tuple( a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ │ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ │ _part_index │ UInt64 │ │ │ │ │ │ 1 │ │ _part_uuid │ UUID │ │ │ │ │ │ 1 │ @@ -104,7 +103,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom │ t │ Tuple( a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ │ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ │ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ │ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ @@ -183,7 +181,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom │ t │ Tuple( a String, b UInt64) │ 0 │ -│ _table │ LowCardinality(String) │ 1 │ │ _part │ LowCardinality(String) │ 1 │ │ _part_index │ UInt64 │ 1 │ │ _part_uuid │ UUID │ 1 │ @@ -222,7 +219,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom │ t │ Tuple( a String, b UInt64) │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ 0 │ 1 │ │ _part │ LowCardinality(String) │ 0 │ 1 │ │ _part_index │ UInt64 │ 0 │ 1 │ │ _part_uuid │ UUID │ 0 │ 1 │ From 8d2ad5383bea6cab2e2107e991f17359cb36ac4f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 14 Feb 2024 15:19:08 +0000 Subject: [PATCH 03/66] Fix execution name for constants --- src/Planner/PlannerActionsVisitor.cpp | 9 +++-- src/Storages/StorageMerge.cpp | 47 ++++++++++++++------------- 2 files changed, 32 insertions(+), 24 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 511e9396a35..8fc200e7d38 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -88,7 +88,10 @@ public: case QueryTreeNodeType::CONSTANT: { const auto & constant_node = node->as(); - result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); + if (constant_node.hasSourceExpression()) + result = calculateActionNodeName(constant_node.getSourceExpression()); + else + result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); break; } case QueryTreeNodeType::FUNCTION: @@ -527,7 +530,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & constant_literal = constant_node.getValue(); const auto & constant_type = constant_node.getResultType(); - auto constant_node_name = calculateConstantActionNodeName(constant_literal, constant_type); + auto constant_node_name = constant_node.hasSourceExpression() + ? action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()) + : calculateConstantActionNodeName(constant_literal, constant_type); ColumnWithTypeAndName column; column.name = constant_node_name; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 79d7b83cada..029ab4d4e4c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1001,34 +1001,37 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( Block pipe_header = builder->getHeader(); - if (has_database_virtual_column && !pipe_header.has("_database")) + if (!allow_experimental_analyzer) { - ColumnWithTypeAndName column; - column.name = "_database"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); + if (has_database_virtual_column && !pipe_header.has("_database")) + { + ColumnWithTypeAndName column; + column.name = "_database"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); + } - if (has_table_virtual_column && !pipe_header.has("_table")) - { - ColumnWithTypeAndName column; - column.name = "_table"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); + if (has_table_virtual_column && !pipe_header.has("_table")) + { + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); + } } /// Subordinary tables could have different but convertible types, like numeric types of different width. From 212245457662ebfbc7a6e865ec9518b028506bef Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 19 Feb 2024 12:41:29 +0000 Subject: [PATCH 04/66] Temporary progress --- src/Analyzer/ConstantNode.cpp | 96 ++++++++++++++------------- src/Analyzer/ConstantNode.h | 2 + src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 46 +++++++++++-- src/Planner/PlannerContext.cpp | 10 ++- src/Planner/PlannerContext.h | 12 +++- src/Storages/StorageMerge.cpp | 55 ++++++++------- 7 files changed, 140 insertions(+), 83 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 69bed3dbe90..ce6da693f93 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -38,52 +38,9 @@ ConstantNode::ConstantNode(Field value_) : ConstantNode(value_, applyVisitor(FieldToDataType(), value_)) {} -void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const -{ - buffer << std::string(indent, ' ') << "CONSTANT id: " << format_state.getNodeId(this); - - if (hasAlias()) - buffer << ", alias: " << getAlias(); - - buffer << ", constant_value: " << constant_value->getValue().dump(); - buffer << ", constant_value_type: " << constant_value->getType()->getName(); - - if (getSourceExpression()) - { - buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; - getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4); - } -} - -bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const -{ - const auto & rhs_typed = assert_cast(rhs); - return *constant_value == *rhs_typed.constant_value && value_string == rhs_typed.value_string; -} - -void ConstantNode::updateTreeHashImpl(HashState & hash_state) const -{ - auto type_name = constant_value->getType()->getName(); - hash_state.update(type_name.size()); - hash_state.update(type_name); - - hash_state.update(value_string.size()); - hash_state.update(value_string); -} - -QueryTreeNodePtr ConstantNode::cloneImpl() const -{ - return std::make_shared(constant_value, source_expression); -} - -ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const +bool ConstantNode::requiresCastCall() const { const auto & constant_value_literal = constant_value->getValue(); - auto constant_value_ast = std::make_shared(constant_value_literal); - - if (!options.add_cast_for_constants) - return constant_value_ast; - bool need_to_add_cast_function = false; auto constant_value_literal_type = constant_value_literal.getType(); WhichDataType constant_value_type(constant_value->getType()); @@ -131,7 +88,56 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const // Add cast if constant was created as a result of constant folding. // Constant folding may lead to type transformation and literal on shard // may have a different type. - if (need_to_add_cast_function || source_expression != nullptr) + return need_to_add_cast_function || source_expression != nullptr; +} + +void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "CONSTANT id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); + + if (getSourceExpression()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; + getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return *constant_value == *rhs_typed.constant_value && value_string == rhs_typed.value_string; +} + +void ConstantNode::updateTreeHashImpl(HashState & hash_state) const +{ + auto type_name = constant_value->getType()->getName(); + hash_state.update(type_name.size()); + hash_state.update(type_name); + + hash_state.update(value_string.size()); + hash_state.update(value_string); +} + +QueryTreeNodePtr ConstantNode::cloneImpl() const +{ + return std::make_shared(constant_value, source_expression); +} + +ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const +{ + const auto & constant_value_literal = constant_value->getValue(); + auto constant_value_ast = std::make_shared(constant_value_literal); + + if (!options.add_cast_for_constants) + return constant_value_ast; + + if (requiresCastCall()) { auto constant_type_name_ast = std::make_shared(constant_value->getType()->getName()); return makeASTFunction("_CAST", std::move(constant_value_ast), std::move(constant_type_name_ast)); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 51c98a4a3b3..c0df092293d 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -75,6 +75,8 @@ public: return constant_value->getType(); } + bool requiresCastCall() const; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; protected: diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index bcc42dbae7f..65033c6f66b 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1153,7 +1153,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node, if (select_query_options.is_subquery) updateContextForSubqueryExecution(mutable_context); - return std::make_shared(mutable_context, std::move(global_planner_context)); + return std::make_shared(mutable_context, std::move(global_planner_context), select_query_options); } Planner::Planner(const QueryTreeNodePtr & query_tree_, diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 8fc200e7d38..b33e1a3509c 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -44,6 +44,22 @@ namespace ErrorCodes namespace { +String calculateActionNodeNameForConstant(const ConstantNode & constant_node) +{ + WriteBufferFromOwnString buffer; + if (constant_node.requiresCastCall()) + buffer << "_CAST("; + + buffer << calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); + + if (constant_node.requiresCastCall()) + { + buffer << ", '" << constant_node.getResultType()->getName() << "'_String)"; + } + + return buffer.str(); +} + class ActionNodeNameHelper { public: @@ -88,10 +104,17 @@ public: case QueryTreeNodeType::CONSTANT: { const auto & constant_node = node->as(); - if (constant_node.hasSourceExpression()) - result = calculateActionNodeName(constant_node.getSourceExpression()); + if (planner_context.isASTLevelOptimizationAllowed()) + { + result = calculateActionNodeNameForConstant(constant_node); + } else - result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); + { + if (constant_node.hasSourceExpression()) + result = calculateActionNodeName(constant_node.getSourceExpression()); + else + result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); + } break; } case QueryTreeNodeType::FUNCTION: @@ -530,9 +553,20 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & constant_literal = constant_node.getValue(); const auto & constant_type = constant_node.getResultType(); - auto constant_node_name = constant_node.hasSourceExpression() - ? action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()) - : calculateConstantActionNodeName(constant_literal, constant_type); + auto constant_node_name = [&]() + { + if (planner_context->isASTLevelOptimizationAllowed()) + { + return calculateActionNodeNameForConstant(constant_node); + } + else + { + if (constant_node.hasSourceExpression()) + return action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()); + else + return calculateConstantActionNodeName(constant_literal, constant_type); + } + }(); ColumnWithTypeAndName column; column.name = constant_node_name; diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 422c8c1d01f..57db84d5031 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include "Interpreters/SelectQueryOptions.h" namespace DB { @@ -41,9 +43,10 @@ bool GlobalPlannerContext::hasColumnIdentifier(const ColumnIdentifier & column_i return column_identifiers.contains(column_identifier); } -PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_) +PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_, const SelectQueryOptions & select_query_options_) : query_context(std::move(query_context_)) , global_planner_context(std::move(global_planner_context_)) + , select_query_options(select_query_options_) {} TableExpressionData & PlannerContext::getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node) @@ -116,6 +119,11 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que return table_expression_data->getColumnIdentifierOrNull(column_name); } +bool PlannerContext::isASTLevelOptimizationAllowed() const +{ + return !(query_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY || select_query_options.ignore_ast_optimizations); +} + PlannerContext::SetKey PlannerContext::createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node) { const auto set_source_hash = set_source_node->getTreeHash(); diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index d7ea4fd95dd..49272429b43 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -48,7 +49,7 @@ class PlannerContext { public: /// Create planner context with query context and global planner context - PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_); + PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_, const SelectQueryOptions & select_query_options_); /// Get planner context query context ContextPtr getQueryContext() const @@ -80,6 +81,11 @@ public: return global_planner_context; } + const SelectQueryOptions & getSelectQueryOptions() const + { + return select_query_options; + } + /// Get or create table expression data for table expression node. TableExpressionData & getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node); @@ -135,6 +141,8 @@ public: static SetKey createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node); PreparedSets & getPreparedSets() { return prepared_sets; } + + bool isASTLevelOptimizationAllowed() const; private: /// Query context ContextMutablePtr query_context; @@ -142,6 +150,8 @@ private: /// Global planner context GlobalPlannerContextPtr global_planner_context; + SelectQueryOptions select_query_options; + /// Column node to column identifier std::unordered_map column_node_to_column_identifier; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 029ab4d4e4c..591a0ae375e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1001,37 +1001,34 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( Block pipe_header = builder->getHeader(); - if (!allow_experimental_analyzer) + if (has_database_virtual_column && !pipe_header.has("_database")) { - if (has_database_virtual_column && !pipe_header.has("_database")) - { - ColumnWithTypeAndName column; - column.name = "_database"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); + ColumnWithTypeAndName column; + column.name = "_database"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); + } - if (has_table_virtual_column && !pipe_header.has("_table")) - { - ColumnWithTypeAndName column; - column.name = "_table"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); + if (has_table_virtual_column && !pipe_header.has("_table")) + { + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); } /// Subordinary tables could have different but convertible types, like numeric types of different width. @@ -1393,7 +1390,7 @@ void ReadFromMerge::convertAndFilterSourceStream( const RowPolicyDataOpt & row_policy_data_opt, ContextMutablePtr local_context, QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage) + QueryProcessingStage::Enum processed_stage [[maybe_unused]]) { Block before_block_header = builder.getHeader(); @@ -1452,9 +1449,9 @@ void ReadFromMerge::convertAndFilterSourceStream( ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; - if (local_context->getSettingsRef().allow_experimental_analyzer - && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) - convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; + // if (local_context->getSettingsRef().allow_experimental_analyzer + // && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) + // convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; if (row_policy_data_opt) { From 601c3c33bc802b6c95ab2371379da4da90bd2baa Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 22 Feb 2024 17:44:41 +0100 Subject: [PATCH 05/66] Set correct execution name for ConstantNode --- src/Analyzer/ConstantNode.cpp | 18 ++++++++++++++++++ src/Analyzer/ConstantNode.h | 2 ++ src/Planner/PlannerActionsVisitor.cpp | 14 ++++++++++++-- .../02227_union_match_by_name.reference | 2 +- 4 files changed, 33 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index ce6da693f93..83446ca7d16 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Analyzer/FunctionNode.h" #include #include @@ -91,6 +92,23 @@ bool ConstantNode::requiresCastCall() const return need_to_add_cast_function || source_expression != nullptr; } +bool ConstantNode::receivedFromInitiatorServer() const +{ + if (!hasSourceExpression()) + return false; + + auto * cast_function = getSourceExpression()->as(); + if (!cast_function || cast_function->getFunctionName() != "_CAST") + return false; + for (auto const & argument : cast_function->getArguments()) + { + auto * constant_arg = argument->as(); + if (!constant_arg || constant_arg->hasSourceExpression()) + return false; + } + return true; +} + void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "CONSTANT id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index c0df092293d..18090c56630 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -77,6 +77,8 @@ public: bool requiresCastCall() const; + bool receivedFromInitiatorServer() const; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; protected: diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index b33e1a3509c..89d843a28ac 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -111,7 +111,12 @@ public: else { if (constant_node.hasSourceExpression()) - result = calculateActionNodeName(constant_node.getSourceExpression()); + { + if (constant_node.receivedFromInitiatorServer()) + result = calculateActionNodeNameForConstant(constant_node); + else + result = calculateActionNodeName(constant_node.getSourceExpression()); + } else result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); } @@ -562,7 +567,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi else { if (constant_node.hasSourceExpression()) - return action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()); + { + if (constant_node.receivedFromInitiatorServer()) + return calculateActionNodeNameForConstant(constant_node); + else + return action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()); + } else return calculateConstantActionNodeName(constant_literal, constant_type); } diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index c28035fab49..d726ae86de7 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -36,7 +36,7 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: x Nullable(Nothing) y UInt8 Expression (Projection) - Header: NULL_Nullable(Nothing) Nullable(Nothing) + Header: _CAST(NULL_Nullable(Nothing), \'Nullable(Nothing)\'_String) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) Header: __table5.dummy UInt8 From 81df9c5a273e2f6533147889c929c4a519dc08af Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 22 Feb 2024 19:07:54 +0100 Subject: [PATCH 06/66] Use possition to merge plans --- src/Storages/StorageMerge.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 591a0ae375e..79d7b83cada 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1390,7 +1390,7 @@ void ReadFromMerge::convertAndFilterSourceStream( const RowPolicyDataOpt & row_policy_data_opt, ContextMutablePtr local_context, QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage [[maybe_unused]]) + QueryProcessingStage::Enum processed_stage) { Block before_block_header = builder.getHeader(); @@ -1449,9 +1449,9 @@ void ReadFromMerge::convertAndFilterSourceStream( ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; - // if (local_context->getSettingsRef().allow_experimental_analyzer - // && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) - // convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; + if (local_context->getSettingsRef().allow_experimental_analyzer + && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) + convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; if (row_policy_data_opt) { From 011c694117845500c82f9563c65930429979982f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 23 Feb 2024 01:34:17 +0100 Subject: [PATCH 07/66] add none_only_active DDL output mode --- docs/en/operations/settings/settings.md | 1 + src/Core/Settings.h | 2 +- src/Core/SettingsEnums.cpp | 1 + src/Core/SettingsEnums.h | 1 + src/Interpreters/executeDDLQueryOnCluster.cpp | 25 ++++++++++++------- ...distributed_ddl_output_mode_long.reference | 4 +-- .../01175_distributed_ddl_output_mode_long.sh | 4 +-- .../02447_drop_database_replica.reference | 2 ++ .../02447_drop_database_replica.sh | 5 ++-- 9 files changed, 29 insertions(+), 16 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b11a04e10ec..d076eeabc9a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3956,6 +3956,7 @@ Possible values: - `none` — Is similar to throw, but distributed DDL query returns no result set. - `null_status_on_timeout` — Returns `NULL` as execution status in some rows of result set instead of throwing `TIMEOUT_EXCEEDED` if query is not finished on the corresponding hosts. - `never_throw` — Do not throw `TIMEOUT_EXCEEDED` and do not rethrow exceptions if query has failed on some hosts. +- `none_only_active` - similar to `none`, but doesn't wait for inactive replicas of the `Replicated` database. Note: with this mode it's impossible to figure out that the query was not executed on some replica and will be executed in background. - `null_status_on_timeout_only_active` — similar to `null_status_on_timeout`, but doesn't wait for inactive replicas of the `Replicated` database - `throw_only_active` — similar to `throw`, but doesn't wait for inactive replicas of the `Replicated` database diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 433195af9c3..b6437e98a85 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -695,7 +695,7 @@ class IColumn; M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(Bool, cloud_mode, false, "Only available in ClickHouse Cloud", 0) \ M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \ - M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result, one of: 'none', 'throw', 'null_status_on_timeout', 'never_throw'", 0) \ + M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result, one of: 'none', 'throw', 'null_status_on_timeout', 'never_throw', 'none_only_active', 'throw_only_active', 'null_status_on_timeout_only_active'", 0) \ M(UInt64, distributed_ddl_entry_format_version, 5, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 0c84c1cc7d2..892d10425ad 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -117,6 +117,7 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS, {"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT}, {"throw_only_active", DistributedDDLOutputMode::THROW_ONLY_ACTIVE}, {"null_status_on_timeout_only_active", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE}, + {"none_only_active", DistributedDDLOutputMode::NONE_ONLY_ACTIVE}, {"never_throw", DistributedDDLOutputMode::NEVER_THROW}}) IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 246cdf6f684..2f29519061f 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -175,6 +175,7 @@ enum class DistributedDDLOutputMode NEVER_THROW, THROW_ONLY_ACTIVE, NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE, + NONE_ONLY_ACTIVE, }; DECLARE_SETTING_ENUM(DistributedDDLOutputMode) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 6122ec6180a..df8236c11f4 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -252,7 +252,8 @@ BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & en auto source = std::make_shared(node_path, entry, context, hosts_to_wait); io.pipeline = QueryPipeline(std::move(source)); - if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE) + if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE || + context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE) io.pipeline.complete(std::make_shared(io.pipeline.getHeader())); return io; @@ -264,7 +265,9 @@ Block DDLQueryStatusSource::getSampleBlock(ContextPtr context_, bool hosts_to_wa auto maybe_make_nullable = [&](const DataTypePtr & type) -> DataTypePtr { - if (output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE) + if (output_mode == DistributedDDLOutputMode::THROW || + output_mode == DistributedDDLOutputMode::NONE || + output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE) return type; return std::make_shared(type); }; @@ -313,14 +316,15 @@ DDLQueryStatusSource::DDLQueryStatusSource( { auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE - || output_mode == DistributedDDLOutputMode::NONE; + || output_mode == DistributedDDLOutputMode::NONE || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; if (hosts_to_wait) { waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end()); is_replicated_database = true; only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || - output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE; + output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE || + output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; } else { @@ -442,14 +446,16 @@ Chunk DDLQueryStatusSource::generate() size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished; size_t num_active_hosts = current_active_hosts.size(); - constexpr auto msg_format = "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. " - "There are {} unfinished hosts ({} of them are currently executing the task), " - "they are going to execute the query in background"; + constexpr auto msg_format = "Distributed DDL task {} is not finished on {} of {} hosts " + "({} of them are currently executing the task, {} are inactive). " + "They are going to execute the query in background. Was waiting for {} seconds{}"; + if (throw_on_timeout) { if (!first_exception) first_exception = std::make_unique(Exception(ErrorCodes::TIMEOUT_EXCEEDED, - msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts)); + msg_format, node_path, num_unfinished_hosts, waiting_hosts.size(), num_active_hosts, offline_hosts.size(), + watch.elapsedSeconds(), stop_waiting_offline_hosts ? "" : ", which is longer than distributed_ddl_task_timeout")); /// For Replicated database print a list of unfinished hosts as well. Will return empty block on next iteration. if (is_replicated_database) @@ -457,7 +463,8 @@ Chunk DDLQueryStatusSource::generate() return {}; } - LOG_INFO(log, msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts); + LOG_INFO(log, msg_format, node_path, num_unfinished_hosts, waiting_hosts.size(), num_active_hosts, offline_hosts.size(), + watch.elapsedSeconds(), stop_waiting_offline_hosts ? "" : "which is longer than distributed_ddl_task_timeout"); return generateChunkWithUnfinishedHosts(); } diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index b9a66a1e1a9..2151328d8b7 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -3,7 +3,7 @@ Received exception from server: Code: 57. Error: Received from localhost:9000. Error: There was an error on [localhost:9000]: Code: 57. Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) (query: create table none on cluster test_shard_localhost (n int) engine=Memory;) Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Distributed DDL task is not finished on 1 of 2 hosts (0 of them are currently executing the task, 0 are inactive). They are going to execute the query in background. Was waiting for seconds, which is longer than distributed_ddl_task_timeout. (TIMEOUT_EXCEEDED) (query: drop table if exists none on cluster test_unavailable_shard;) throw localhost 9000 0 0 0 @@ -12,7 +12,7 @@ Code: 57. Error: Received from localhost:9000. Error: There was an error on [loc (query: create table throw on cluster test_shard_localhost (n int) engine=Memory format Null;) localhost 9000 0 1 0 Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Distributed DDL task is not finished on 1 of 2 hosts (0 of them are currently executing the task, 0 are inactive). They are going to execute the query in background. Was waiting for seconds, which is longer than distributed_ddl_task_timeout. (TIMEOUT_EXCEEDED) (query: drop table if exists throw on cluster test_unavailable_shard;) null_status_on_timeout localhost 9000 0 0 0 diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh index 12e142adda9..f17e85da60a 100755 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh @@ -54,14 +54,14 @@ $CLIENT --distributed_ddl_output_mode=none -q "create table none on cluster test $CLIENT --distributed_ddl_output_mode=none -q "create table none on cluster test_shard_localhost (n int) engine=Memory;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" # Timeout -run_until_out_contains 'There are 1 unfinished hosts' $CLICKHOUSE_CLIENT_WITH_SETTINGS --distributed_ddl_output_mode=none -q "drop table if exists none on cluster test_unavailable_shard;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" | sed "s/Watching task .* is executing longer/Watching task is executing longer/" +run_until_out_contains 'not finished on 1 ' $CLICKHOUSE_CLIENT_WITH_SETTINGS --distributed_ddl_output_mode=none -q "drop table if exists none on cluster test_unavailable_shard;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" | sed "s/Distributed DDL task .* is not finished/Distributed DDL task is not finished/" | sed "s/for .* seconds/for seconds/" $CLIENT --distributed_ddl_output_mode=throw -q "select value from system.settings where name='distributed_ddl_output_mode';" $CLIENT --distributed_ddl_output_mode=throw -q "create table throw on cluster test_shard_localhost (n int) engine=Memory;" $CLIENT --distributed_ddl_output_mode=throw -q "create table throw on cluster test_shard_localhost (n int) engine=Memory format Null;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" -run_until_out_contains 'There are 1 unfinished hosts' $CLICKHOUSE_CLIENT_WITH_SETTINGS --distributed_ddl_output_mode=throw -q "drop table if exists throw on cluster test_unavailable_shard;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" | sed "s/Watching task .* is executing longer/Watching task is executing longer/" +run_until_out_contains 'not finished on 1 ' $CLICKHOUSE_CLIENT_WITH_SETTINGS --distributed_ddl_output_mode=throw -q "drop table if exists throw on cluster test_unavailable_shard;" 2>&1 | sed "s/DB::Exception/Error/g" | sed "s/ (version.*)//" | sed "s/Distributed DDL task .* is not finished/Distributed DDL task is not finished/" | sed "s/for .* seconds/for seconds/" $CLIENT --distributed_ddl_output_mode=null_status_on_timeout -q "select value from system.settings where name='distributed_ddl_output_mode';" diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index 1af3ee244f1..bd3b689ca3c 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -13,6 +13,7 @@ t rdb_default 1 1 s1 r1 1 2 2 +2 s1 r1 OK 2 0 s1 r2 QUEUED 2 0 s2 r1 QUEUED 2 0 @@ -24,4 +25,5 @@ rdb_default 1 2 s1 r2 0 t t2 t3 +t4 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index fb89db5045b..5c4604bc8cd 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -33,8 +33,9 @@ $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_na $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" # Also check that it doesn't exceed distributed_ddl_task_timeout waiting for inactive replicas -timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" -timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t3 (n int) engine=Log" | sort +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=none_only_active -q "create table $db.t2 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t3 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t4 (n int) engine=Log" | sort $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" From d13b2a91c1bfbb7f59b5380ceb1a5057e88043da Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 23 Feb 2024 13:33:12 +0000 Subject: [PATCH 08/66] Fix virtual tables --- src/Interpreters/ActionsDAG.cpp | 2 +- src/Storages/StorageMerge.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 7240679abb7..7412eea5e32 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1318,7 +1318,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( size_t num_result_columns = result.size(); if (mode == MatchColumnsMode::Position && num_input_columns != num_result_columns) - throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match"); + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match (source: {} and result: {})", num_input_columns, num_result_columns); if (add_casted_columns && mode != MatchColumnsMode::Name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 79d7b83cada..5b18fb8f838 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1001,7 +1001,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( Block pipe_header = builder->getHeader(); - if (has_database_virtual_column && !pipe_header.has("_database")) + if (has_database_virtual_column && common_header.has("_database") && !pipe_header.has("_database")) { ColumnWithTypeAndName column; column.name = "_database"; @@ -1016,7 +1016,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { return std::make_shared(stream_header, adding_column_actions); }); } - if (has_table_virtual_column && !pipe_header.has("_table")) + if (has_table_virtual_column && common_header.has("_table") && !pipe_header.has("_table")) { ColumnWithTypeAndName column; column.name = "_table"; From a2cabc9fe13c0227db5f2bd2eaf090aa9b0189a5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 27 Feb 2024 16:16:15 +0100 Subject: [PATCH 09/66] Fixup --- src/Analyzer/ConstantNode.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 83446ca7d16..e3b87edbdc6 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -100,12 +100,6 @@ bool ConstantNode::receivedFromInitiatorServer() const auto * cast_function = getSourceExpression()->as(); if (!cast_function || cast_function->getFunctionName() != "_CAST") return false; - for (auto const & argument : cast_function->getArguments()) - { - auto * constant_arg = argument->as(); - if (!constant_arg || constant_arg->hasSourceExpression()) - return false; - } return true; } From 9b3a0273c720112c04b018b2a9aec0ac1765c1d1 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 29 Feb 2024 17:11:08 +0800 Subject: [PATCH 10/66] Update settings.md by adding some missing settings. --- docs/zh/operations/settings/settings.md | 37 +++++++++++++++++++++++-- 1 file changed, 35 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 1874970ac95..69f76a35edc 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -649,11 +649,22 @@ log_query_threads=1 ## max_query_size {#settings-max_query_size} -查询的最大部分,可以被带到RAM用于使用SQL解析器进行解析。 -插入查询还包含由单独的流解析器(消耗O(1)RAM)处理的插入数据,这些数据不包含在此限制中。 +SQL 解析器解析的查询字符串的最大字节数。 INSERT 查询的 VALUES 子句中的数据由单独的流解析器(消耗 O(1) RAM)处理,并且不受此限制的影响。 默认值:256KiB。 + +## max_parser_depth {#max_parser_depth} + +限制递归下降解析器中的最大递归深度。允许控制堆栈大小。 + +可能的值: + +- 正整数。 +- 0 — 递归深度不受限制。 + +默认值:1000。 + ## interactive_delay {#interactive-delay} 以微秒为单位的间隔,用于检查请求执行是否已被取消并发送进度。 @@ -1064,6 +1075,28 @@ ClickHouse生成异常 默认值:0。 +## optimize_functions_to_subcolumns {#optimize_functions_to_subcolumns} + +启用或禁用通过将某些函数转换为读取子列的优化。这减少了要读取的数据量。 + +这些函数可以转化为: + +- [length](../../sql-reference/functions/array-functions.md/#array_functions-length) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 +- [empty](../../sql-reference/functions/array-functions.md/#function-empty) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 +- [notEmpty](../../sql-reference/functions/array-functions.md/#function-notempty) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 +- [isNull](../../sql-reference/operators/index.md#operator-is-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 +- [isNotNull](../../sql-reference/operators/index.md#is-not-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 +- [count](../../sql-reference/aggregate-functions/reference/count.md) 读取 [null](../../sql-reference/data-types/nullable.md/ #finding-null) 子列。 +- [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) 读取 [keys](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 +- [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) 读取 [values](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 + +可能的值: + +- 0 — 禁用优化。 +- 1 — 优化已启用。 + +默认值:`0`。 + ## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - 类型:秒 From 7116dd7dca60e05234b1edf04396e9ec492d0b08 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 29 Feb 2024 17:22:25 +0800 Subject: [PATCH 11/66] Update array.md by adding `Array Size` part --- docs/zh/sql-reference/data-types/array.md | 26 ++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/data-types/array.md b/docs/zh/sql-reference/data-types/array.md index 46c40b889ad..41ed98fd053 100644 --- a/docs/zh/sql-reference/data-types/array.md +++ b/docs/zh/sql-reference/data-types/array.md @@ -1,7 +1,7 @@ --- slug: /zh/sql-reference/data-types/array --- -# 阵列(T) {#data-type-array} +# 数组(T) {#data-type-array} 由 `T` 类型元素组成的数组。 @@ -66,3 +66,27 @@ SELECT array(1, 'a') Received exception from server (version 1.1.54388): Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. ``` + +## 数组大小 {#array-size} + +可以使用`size0`子列找到数组的大小,而无需读取整个列。对于多维数组,您可以使用`sizeN-1`,其中`N`是所需的维度。 + +**例子** + +SQL查询: + +```sql +CREATE TABLE t_arr (`arr` Array(Array(Array(UInt32)))) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_arr VALUES ([[[12, 13, 0, 1],[12]]]); + +SELECT arr.size0, arr.size1, arr.size2 FROM t_arr; +``` + +结果: + +``` text +┌─arr.size0─┬─arr.size1─┬─arr.size2─┐ +│ 1 │ [2] │ [[4,1]] │ +└───────────┴───────────┴───────────┘ +``` From ea9ef507fcca531d470a62462e571051ec7bf633 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 29 Feb 2024 17:29:37 +0800 Subject: [PATCH 12/66] Update nullable.md by adding #finding-null --- docs/zh/sql-reference/data-types/nullable.md | 28 ++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/docs/zh/sql-reference/data-types/nullable.md b/docs/zh/sql-reference/data-types/nullable.md index 94311f8298a..7ecbc3d1f40 100644 --- a/docs/zh/sql-reference/data-types/nullable.md +++ b/docs/zh/sql-reference/data-types/nullable.md @@ -20,6 +20,34 @@ slug: /zh/sql-reference/data-types/nullable 掩码文件中的条目允许ClickHouse区分每个表行的对应数据类型的«NULL»和默认值由于有额外的文件,«Nullable»列比普通列消耗更多的存储空间 +## null子列 {#finding-null} + +It is possible to find `NULL` values in a column by using `null` subcolumn without reading the whole column. It returns `1` if the corresponding value is `NULL` and `0` otherwise. +通过使用`null`子列可以在列中查找`NULL`值,而无需读取整个列。如果对应的值为`NULL`,则返回`1`,否则返回`0`。 + +**示例** + +SQL查询: + +``` sql +CREATE TABLE nullable (`n` Nullable(UInt32)) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO nullable VALUES (1) (NULL) (2) (NULL); + +SELECT n.null FROM nullable; +``` + +结果: + +``` text +┌─n.null─┐ +│ 0 │ +│ 1 │ +│ 0 │ +│ 1 │ +└────────┘ +``` + ## 用法示例 {#yong-fa-shi-li} ``` sql From 6ea5fabac51ba5ab52276f799a69e7bfe0ddcc09 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 29 Feb 2024 17:39:07 +0800 Subject: [PATCH 13/66] Update settings.md --- docs/zh/operations/settings/settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 69f76a35edc..307159aa5a1 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -1082,11 +1082,11 @@ ClickHouse生成异常 这些函数可以转化为: - [length](../../sql-reference/functions/array-functions.md/#array_functions-length) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 -- [empty](../../sql-reference/functions/array-functions.md/#function-empty) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 -- [notEmpty](../../sql-reference/functions/array-functions.md/#function-notempty) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 +- [empty](../../sql-reference/functions/array-functions.md/#empty函数) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 +- [notEmpty](../../sql-reference/functions/array-functions.md/#notempty函数) 读取 [size0](../../sql-reference/data-types/array.md/#array-size)子列。 - [isNull](../../sql-reference/operators/index.md#operator-is-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 - [isNotNull](../../sql-reference/operators/index.md#is-not-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 -- [count](../../sql-reference/aggregate-functions/reference/count.md) 读取 [null](../../sql-reference/data-types/nullable.md/ #finding-null) 子列。 +- [count](../../sql-reference/aggregate-functions/reference/count.md) 读取 [null](../../sql-reference/data-types/nullable.md/#finding-null) 子列。 - [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) 读取 [keys](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 - [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) 读取 [values](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 From da0cafc31c5dd08057969086ede0f9495daa7b31 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 29 Feb 2024 13:30:55 -0800 Subject: [PATCH 14/66] Fix arm64 floating point result inconsistency issue --- cmake/linux/toolchain-aarch64.cmake | 4 ++-- tests/queries/0_stateless/02813_seriesDecomposeSTL.sql | 3 --- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index b80cc01296d..d2ce2d97d8e 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -13,6 +13,6 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -ffp-contract=off --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -ffp-contract=off --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") diff --git a/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql b/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql index 929d0474e09..496267f2476 100644 --- a/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql +++ b/tests/queries/0_stateless/02813_seriesDecomposeSTL.sql @@ -1,6 +1,3 @@ --- Tags: no-cpu-aarch64 --- Tag no-cpu-aarch64: values generated are slighly different on aarch64 - DROP TABLE IF EXISTS tb2; CREATE TABLE tb2 (`period` UInt32, `ts` Array(Float64)) ENGINE = Memory; From 9c5ed092f8bbb4e5d5dad5b3819879ed6fa04be1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 4 Mar 2024 14:15:07 +0100 Subject: [PATCH 15/66] Fix CREATE VIEW with scalar subquery #ci_set_analyzer --- src/Planner/PlannerActionsVisitor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 89d843a28ac..e5610dd6fe7 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -110,7 +110,7 @@ public: } else { - if (constant_node.hasSourceExpression()) + if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY) { if (constant_node.receivedFromInitiatorServer()) result = calculateActionNodeNameForConstant(constant_node); @@ -566,7 +566,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } else { - if (constant_node.hasSourceExpression()) + if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY) { if (constant_node.receivedFromInitiatorServer()) return calculateActionNodeNameForConstant(constant_node); From bf91fe8bae6680d0bf68b3917a253c2316f4142b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 4 Mar 2024 20:06:46 +0100 Subject: [PATCH 16/66] fix --- tests/integration/test_replicated_database/test.py | 2 +- .../integration/test_replicated_database_cluster_groups/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index b47f86a843d..4fa42ff6f1a 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -509,7 +509,7 @@ def test_alters_from_different_replicas(started_cluster): settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently executing the task" + "is not finished on 1 of 3 hosts" in competing_node.query_and_get_error( "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", settings=settings, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index 647626d8014..b501c2573b2 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -96,7 +96,7 @@ def test_cluster_groups(started_cluster): main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently executing the task)" + "is not finished on 1 of 2 hosts" in main_node_1.query_and_get_error( "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", settings=settings, From 8be4a115ac5f5cbe7bb7dac46a03c83e4aca8bee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 4 Mar 2024 19:22:18 +0000 Subject: [PATCH 17/66] Automatic style fix --- tests/integration/test_replicated_database/test.py | 9 +++------ .../test_replicated_database_cluster_groups/test.py | 9 +++------ 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 4fa42ff6f1a..bd80ac25468 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -508,12 +508,9 @@ def test_alters_from_different_replicas(started_cluster): dummy_node.stop_clickhouse(kill=True) settings = {"distributed_ddl_task_timeout": 5} - assert ( - "is not finished on 1 of 3 hosts" - in competing_node.query_and_get_error( - "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", - settings=settings, - ) + assert "is not finished on 1 of 3 hosts" in competing_node.query_and_get_error( + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", + settings=settings, ) settings = { "distributed_ddl_task_timeout": 5, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index b501c2573b2..91361c1850b 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -95,12 +95,9 @@ def test_cluster_groups(started_cluster): # Exception main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} - assert ( - "is not finished on 1 of 2 hosts" - in main_node_1.query_and_get_error( - "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", - settings=settings, - ) + assert "is not finished on 1 of 2 hosts" in main_node_1.query_and_get_error( + "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", + settings=settings, ) # 3. After start both groups are synced From 51873f0dc9d45dc6993ffdd8379727761c168cb2 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 4 Mar 2024 11:48:12 -0800 Subject: [PATCH 18/66] Just triggering build, no changes From 18b17523e403998fdaaedbfb08736de3a20b38a5 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Tue, 5 Mar 2024 15:32:17 +0800 Subject: [PATCH 19/66] Move connection drain from prepare to work --- src/Processors/Sources/RemoteSource.cpp | 18 ++++++++++++++++-- src/Processors/Sources/RemoteSource.h | 3 +++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index dfa311a7079..c09657ece35 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -72,19 +72,33 @@ ISource::Status RemoteSource::prepare() if (is_async_state) return Status::Async; + if (executor_finished) + return Status::Finished; + Status status = ISource::prepare(); /// To avoid resetting the connection (because of "unfinished" query) in the /// RemoteQueryExecutor it should be finished explicitly. if (status == Status::Finished) { - query_executor->finish(); is_async_state = false; - return status; + need_drain = true; + return Status::Ready; } return status; } +void RemoteSource::work() +{ + if (need_drain) + { + query_executor->finish(); + executor_finished = true; + return; + } + ISource::work(); +} + std::optional RemoteSource::tryGenerate() { /// onCancel() will do the cancel if the query was sent. diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index dbfa0156331..80d84068e39 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -23,6 +23,7 @@ public: ~RemoteSource() override; Status prepare() override; + void work() override; String getName() const override { return "Remote"; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } @@ -40,6 +41,8 @@ protected: private: bool was_query_sent = false; + bool need_drain = false; + bool executor_finished = false; bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; From 5524f5901f4c22e7e97e1120410e7f47577e07e1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 5 Mar 2024 16:18:03 +0100 Subject: [PATCH 20/66] Cleanup code --- src/Analyzer/ConstantNode.cpp | 3 +- src/Analyzer/ConstantNode.h | 2 + src/Planner/PlannerActionsVisitor.cpp | 69 +++++++++++++++++++++++++-- src/Planner/PlannerContext.cpp | 8 +--- src/Planner/PlannerContext.h | 13 +++-- 5 files changed, 75 insertions(+), 20 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index e3b87edbdc6..b6940271b1e 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -1,9 +1,10 @@ #include +#include + #include #include #include -#include "Analyzer/FunctionNode.h" #include #include diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 18090c56630..45f85cec4a3 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -75,8 +75,10 @@ public: return constant_value->getType(); } + /// Check if convertation to AST requires wrapping with _CAST function. bool requiresCastCall() const; + /// Check if constant is a result of _CAST function constant folding. bool receivedFromInitiatorServer() const; void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index e5610dd6fe7..52971539dd7 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -44,7 +44,12 @@ namespace ErrorCodes namespace { -String calculateActionNodeNameForConstant(const ConstantNode & constant_node) +/* Calculates Action node name for ConstantNode. + * + * If converting to AST will add a '_CAST' function call, + * the result action name will also include it. + */ +String calculateActionNodeNameWithCastIfNeeded(const ConstantNode & constant_node) { WriteBufferFromOwnString buffer; if (constant_node.requiresCastCall()) @@ -104,16 +109,43 @@ public: case QueryTreeNodeType::CONSTANT: { const auto & constant_node = node->as(); + /* To ensure that headers match during distributed query we need to simulate action node naming on + * secondary servers. If we don't do that headers will mismatch due to constant folding. + * + * +--------+ + * -----------------| Server |---------------- + * / +--------+ \ + * / \ + * v v + * +-----------+ +-----------+ + * | Initiator | ------ | Secondary |------ + * +-----------+ / +-----------+ \ + * | / \ + * | / \ + * v / \ + * +---------------+ v v + * | Wrap in _CAST | +----------------------------+ +----------------------+ + * | if needed | | Constant folded from _CAST | | Constant folded from | + * +---------------+ +----------------------------+ | another expression | + * | +----------------------+ + * v | + * +----------------------------+ v + * | Name ConstantNode the same | +--------------------------+ + * | as on initiator server | | Generate action name for | + * | (wrap in _CAST if needed) | | original expression | + * +----------------------------+ +--------------------------+ + */ if (planner_context.isASTLevelOptimizationAllowed()) { - result = calculateActionNodeNameForConstant(constant_node); + result = calculateActionNodeNameWithCastIfNeeded(constant_node); } else { + // Need to check if constant folded from QueryNode until https://github.com/ClickHouse/ClickHouse/issues/60847 is fixed. if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY) { if (constant_node.receivedFromInitiatorServer()) - result = calculateActionNodeNameForConstant(constant_node); + result = calculateActionNodeNameWithCastIfNeeded(constant_node); else result = calculateActionNodeName(constant_node.getSourceExpression()); } @@ -560,16 +592,43 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto constant_node_name = [&]() { + /* To ensure that headers match during distributed query we need to simulate action node naming on + * secondary servers. If we don't do that headers will mismatch due to constant folding. + * + * +--------+ + * -----------------| Server |---------------- + * / +--------+ \ + * / \ + * v v + * +-----------+ +-----------+ + * | Initiator | ------ | Secondary |------ + * +-----------+ / +-----------+ \ + * | / \ + * | / \ + * v / \ + * +---------------+ v v + * | Wrap in _CAST | +----------------------------+ +----------------------+ + * | if needed | | Constant folded from _CAST | | Constant folded from | + * +---------------+ +----------------------------+ | another expression | + * | +----------------------+ + * v | + * +----------------------------+ v + * | Name ConstantNode the same | +--------------------------+ + * | as on initiator server | | Generate action name for | + * | (wrap in _CAST if needed) | | original expression | + * +----------------------------+ +--------------------------+ + */ if (planner_context->isASTLevelOptimizationAllowed()) { - return calculateActionNodeNameForConstant(constant_node); + return calculateActionNodeNameWithCastIfNeeded(constant_node); } else { + // Need to check if constant folded from QueryNode until https://github.com/ClickHouse/ClickHouse/issues/60847 is fixed. if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY) { if (constant_node.receivedFromInitiatorServer()) - return calculateActionNodeNameForConstant(constant_node); + return calculateActionNodeNameWithCastIfNeeded(constant_node); else return action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression()); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 57db84d5031..f33255f0a44 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -4,7 +4,6 @@ #include #include #include -#include "Interpreters/SelectQueryOptions.h" namespace DB { @@ -46,7 +45,7 @@ bool GlobalPlannerContext::hasColumnIdentifier(const ColumnIdentifier & column_i PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_, const SelectQueryOptions & select_query_options_) : query_context(std::move(query_context_)) , global_planner_context(std::move(global_planner_context_)) - , select_query_options(select_query_options_) + , is_ast_level_optimization_allowed(!(query_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY || select_query_options_.ignore_ast_optimizations)) {} TableExpressionData & PlannerContext::getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node) @@ -119,11 +118,6 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que return table_expression_data->getColumnIdentifierOrNull(column_name); } -bool PlannerContext::isASTLevelOptimizationAllowed() const -{ - return !(query_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY || select_query_options.ignore_ast_optimizations); -} - PlannerContext::SetKey PlannerContext::createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node) { const auto set_source_hash = set_source_node->getTreeHash(); diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index e7fa7abf2b6..4d9ba037cac 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -111,11 +111,6 @@ public: return global_planner_context; } - const SelectQueryOptions & getSelectQueryOptions() const - { - return select_query_options; - } - /// Get or create table expression data for table expression node. TableExpressionData & getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node); @@ -172,7 +167,11 @@ public: PreparedSets & getPreparedSets() { return prepared_sets; } - bool isASTLevelOptimizationAllowed() const; + /// Returns false if any of following conditions met: + /// 1. Query is executed on a follower node. + /// 2. ignore_ast_optimizations is set. + bool isASTLevelOptimizationAllowed() const { return is_ast_level_optimization_allowed; } + private: /// Query context ContextMutablePtr query_context; @@ -180,7 +179,7 @@ private: /// Global planner context GlobalPlannerContextPtr global_planner_context; - SelectQueryOptions select_query_options; + bool is_ast_level_optimization_allowed; /// Column node to column identifier std::unordered_map column_node_to_column_identifier; From 55c028322dfb4111baf8d0ad8eac4d48975dd349 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 5 Mar 2024 16:59:22 +0100 Subject: [PATCH 21/66] Fix typo --- src/Analyzer/ConstantNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 45f85cec4a3..24f0c786980 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -75,7 +75,7 @@ public: return constant_value->getType(); } - /// Check if convertation to AST requires wrapping with _CAST function. + /// Check if conversion to AST requires wrapping with _CAST function. bool requiresCastCall() const; /// Check if constant is a result of _CAST function constant folding. From d0e887f8344a8ba70fff50a5d4f23e1a724c3c1b Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Wed, 6 Mar 2024 15:02:18 +0000 Subject: [PATCH 22/66] Update index.md --- .../sql-reference/window-functions/index.md | 108 ++++++++++-------- 1 file changed, 59 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 6340c369bff..5bfe22d23a2 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -5,7 +5,12 @@ sidebar_label: Window Functions title: Window Functions --- -ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported: +Windows functions let you perform calculations across a set of rows that are related to the current row. +Some of the calculations that you can do are similar to those that can be done with an aggregate function, but a window function doesn't cause rows to be grouped into a single output - the individual rows are still returned. + +## Standard Window Functionos + +ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported. | Feature | Support or workaround | |------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| @@ -25,6 +30,8 @@ ClickHouse supports the standard grammar for defining windows and window functio ## ClickHouse-specific Window Functions +There are also the following window function that's specific to ClickHouse: + ### nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL X UNITS]) Finds non-negative derivative for given `metric_column` by `timestamp_column`. @@ -33,40 +40,6 @@ The computed value is the following for each row: - `0` for 1st row, - ${metric_i - metric_{i-1} \over timestamp_i - timestamp_{i-1}} * interval$ for $i_th$ row. -## References - -### GitHub Issues - -The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097). - -All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. - -### Tests - -These tests contain the examples of the currently supported grammar: - -https://github.com/ClickHouse/ClickHouse/blob/master/tests/performance/window_functions.xml - -https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/01591_window_functions.sql - -### Postgres Docs - -https://www.postgresql.org/docs/current/sql-select.html#SQL-WINDOW - -https://www.postgresql.org/docs/devel/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS - -https://www.postgresql.org/docs/devel/functions-window.html - -https://www.postgresql.org/docs/devel/tutorial-window.html - -### MySQL Docs - -https://dev.mysql.com/doc/refman/8.0/en/window-function-descriptions.html - -https://dev.mysql.com/doc/refman/8.0/en/window-functions-usage.html - -https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html - ## Syntax ```text @@ -80,20 +53,7 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] - `PARTITION BY` - defines how to break a resultset into groups. - `ORDER BY` - defines how to order rows inside the group during calculation aggregate_function. - `ROWS or RANGE` - defines bounds of a frame, aggregate_function is calculated within a frame. -- `WINDOW` - allows to reuse a window definition with multiple expressions. - -### Functions - -These functions can be used only as a window function. - -- `row_number()` - Number the current row within its partition starting from 1. -- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. -- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. -- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. -- `rank()` - Rank the current row within its partition with gaps. -- `dense_rank()` - Rank the current row within its partition without gaps. -- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. -- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. +- `WINDOW` - allows multiple expressions to use the same window definition. ```text PARTITION @@ -112,8 +72,23 @@ These functions can be used only as a window function. └─────────────────┘ <--- UNBOUNDED FOLLOWING (END of the PARTITION) ``` +### Functions + +These functions can be used only as a window function. + +- `row_number()` - Number the current row within its partition starting from 1. +- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. +- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. +- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. +- `rank()` - Rank the current row within its partition with gaps. +- `dense_rank()` - Rank the current row within its partition without gaps. +- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. +- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. + ## Examples +Let's have a look at some examples of how window functions can be used. + ```sql CREATE TABLE wf_partition ( @@ -589,6 +564,41 @@ ORDER BY └──────────────┴─────────────────────┴───────┴─────────────────────────┘ ``` +## References + +### GitHub Issues + +The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097). + +All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. + +### Tests + +These tests contain the examples of the currently supported grammar: + +https://github.com/ClickHouse/ClickHouse/blob/master/tests/performance/window_functions.xml + +https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/01591_window_functions.sql + +### Postgres Docs + +https://www.postgresql.org/docs/current/sql-select.html#SQL-WINDOW + +https://www.postgresql.org/docs/devel/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS + +https://www.postgresql.org/docs/devel/functions-window.html + +https://www.postgresql.org/docs/devel/tutorial-window.html + +### MySQL Docs + +https://dev.mysql.com/doc/refman/8.0/en/window-function-descriptions.html + +https://dev.mysql.com/doc/refman/8.0/en/window-functions-usage.html + +https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html + + ## Related Content - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) From 43d5338f9233f19f774293f59cafb0ff2093030e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Mar 2024 17:10:26 +0100 Subject: [PATCH 23/66] restart ci From 0dea920a06d32acdda751cab20ef475163556769 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Mar 2024 18:26:16 +0100 Subject: [PATCH 24/66] reload checks From dbdaa9863e4528e578315531ce3725bb09973901 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 6 Mar 2024 15:54:04 -0800 Subject: [PATCH 25/66] Refactor PR according to review --- CMakeLists.txt | 4 ++-- cmake/linux/toolchain-aarch64.cmake | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 3bd179a799c..49dc12b89d8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -319,8 +319,8 @@ if (COMPILER_CLANG) endif() endif () -set (COMPILER_FLAGS "${COMPILER_FLAGS}") - +# Disable floating-point expression contraction in order to get consistent floating point calculation results across platforms +set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffp-contract=off") # Our built-in unwinder only supports DWARF version up to 4. set (DEBUG_INFO_FLAGS "-g") diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index d2ce2d97d8e..b80cc01296d 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -13,6 +13,6 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -ffp-contract=off --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -ffp-contract=off --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") From dd5510674e43b7ab6ecccd2e7be23f6c56097d74 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 7 Mar 2024 17:17:47 +0800 Subject: [PATCH 26/66] Update docs/zh/sql-reference/data-types/array.md Co-authored-by: flynn --- docs/zh/sql-reference/data-types/array.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/data-types/array.md b/docs/zh/sql-reference/data-types/array.md index 41ed98fd053..da4cea65101 100644 --- a/docs/zh/sql-reference/data-types/array.md +++ b/docs/zh/sql-reference/data-types/array.md @@ -69,7 +69,7 @@ Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception ## 数组大小 {#array-size} -可以使用`size0`子列找到数组的大小,而无需读取整个列。对于多维数组,您可以使用`sizeN-1`,其中`N`是所需的维度。 +可以使用 `size0` 子列找到数组的大小,而无需读取整个列。对于多维数组,您可以使用 `sizeN-1`,其中 `N` 是所需的维度。 **例子** From 8e4dec32058615e2167209934422b9382ca8dca2 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 7 Mar 2024 17:23:05 +0800 Subject: [PATCH 27/66] Update nullable.md --- docs/zh/sql-reference/data-types/nullable.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/zh/sql-reference/data-types/nullable.md b/docs/zh/sql-reference/data-types/nullable.md index 7ecbc3d1f40..b1cc9dd7bae 100644 --- a/docs/zh/sql-reference/data-types/nullable.md +++ b/docs/zh/sql-reference/data-types/nullable.md @@ -22,8 +22,7 @@ slug: /zh/sql-reference/data-types/nullable ## null子列 {#finding-null} -It is possible to find `NULL` values in a column by using `null` subcolumn without reading the whole column. It returns `1` if the corresponding value is `NULL` and `0` otherwise. -通过使用`null`子列可以在列中查找`NULL`值,而无需读取整个列。如果对应的值为`NULL`,则返回`1`,否则返回`0`。 +通过使用 `null` 子列可以在列中查找 `NULL` 值,而无需读取整个列。如果对应的值为 `NULL`,则返回 `1`,否则返回 `0`。 **示例** From b1b999f1c89d213164ba20b01c1a37c16e93d068 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Mar 2024 12:44:57 +0100 Subject: [PATCH 28/66] fix typo --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 5bfe22d23a2..9b2ded7b6ce 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -8,7 +8,7 @@ title: Window Functions Windows functions let you perform calculations across a set of rows that are related to the current row. Some of the calculations that you can do are similar to those that can be done with an aggregate function, but a window function doesn't cause rows to be grouped into a single output - the individual rows are still returned. -## Standard Window Functionos +## Standard Window Functions ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported. From a23d0bb75b85ca371c5d32e977ba8dc861406e8d Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Thu, 7 Mar 2024 21:06:41 +0800 Subject: [PATCH 29/66] Update settings.md From 5b28614f4ccda9a4620d4121479b3328d5aea04f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 7 Mar 2024 14:50:37 +0100 Subject: [PATCH 30/66] Fix TableFunctionExecutable::skipAnalysisForArguments --- src/TableFunctions/TableFunctionExecutable.cpp | 3 +++ .../03006_analyzer_executable_table_function.reference | 0 .../0_stateless/03006_analyzer_executable_table_function.sql | 4 ++++ 3 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03006_analyzer_executable_table_function.reference create mode 100644 tests/queries/0_stateless/03006_analyzer_executable_table_function.sql diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 9fa5ddf15c2..2c3802e8667 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -71,6 +71,9 @@ std::vector TableFunctionExecutable::skipAnalysisForArguments(const Quer const auto & table_function_node_arguments = table_function_node.getArguments().getNodes(); size_t table_function_node_arguments_size = table_function_node_arguments.size(); + if (table_function_node_arguments_size <= 2) + return {}; + std::vector result_indexes; result_indexes.reserve(table_function_node_arguments_size - 2); for (size_t i = 2; i < table_function_node_arguments_size; ++i) diff --git a/tests/queries/0_stateless/03006_analyzer_executable_table_function.reference b/tests/queries/0_stateless/03006_analyzer_executable_table_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03006_analyzer_executable_table_function.sql b/tests/queries/0_stateless/03006_analyzer_executable_table_function.sql new file mode 100644 index 00000000000..4edbcc97f50 --- /dev/null +++ b/tests/queries/0_stateless/03006_analyzer_executable_table_function.sql @@ -0,0 +1,4 @@ +SELECT + toFixedString(toFixedString(toLowCardinality(toFixedString('--------------------', toNullable(20))), toLowCardinality(20)), 20), + * +FROM executable('data String', SETTINGS max_command_execution_time = 100); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From 3a4ef70a208ca4452f646bb78f1fb96a12bc377e Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 7 Mar 2024 06:03:33 -0800 Subject: [PATCH 31/66] Make better looking --- CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 49dc12b89d8..b55e9810361 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -321,6 +321,7 @@ endif () # Disable floating-point expression contraction in order to get consistent floating point calculation results across platforms set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffp-contract=off") + # Our built-in unwinder only supports DWARF version up to 4. set (DEBUG_INFO_FLAGS "-g") From 1b2357198d4c1b92fbe3f5a6c0c11be6f1d80a8f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Mar 2024 17:03:43 +0000 Subject: [PATCH 32/66] Fix: parallel replicas + prewhere --- src/Interpreters/InterpreterSelectQuery.cpp | 5 ++-- ...03006_parallel_replicas_prewhere.reference | 0 .../03006_parallel_replicas_prewhere.sql | 29 +++++++++++++++++++ 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference create mode 100644 tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index eaa256a16b0..80e2d5afef7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2112,7 +2112,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. analysis.prewhere_info - = std::make_shared(std::move(analysis.filter_info->actions), std::move(analysis.filter_info->column_name)); + = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); analysis.prewhere_info->prewhere_actions->projectInput(false); analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; analysis.prewhere_info->need_filter = true; @@ -2121,8 +2121,9 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis } else { + chassert(analysis.filter_info->actions); /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.prewhere_info->row_level_filter->projectInput(false); analysis.filter_info = nullptr; diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql new file mode 100644 index 00000000000..819526b5596 --- /dev/null +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -0,0 +1,29 @@ +DROP POLICY IF EXISTS url_na_log_policy0 ON url_na_log; +DROP TABLE IF EXISTS url_na_log; + +CREATE TABLE url_na_log +( + `SiteId` UInt32, + `DateVisit` Date +) +ENGINE = MergeTree +PRIMARY KEY SiteId +ORDER BY (SiteId, DateVisit) +SETTINGS index_granularity = 1000, min_bytes_for_wide_part = 0; + +CREATE ROW POLICY url_na_log_policy0 ON url_na_log FOR SELECT USING (DateVisit < '2022-08-11') OR (DateVisit > '2022-08-19') TO default; + +INSERT INTO url_na_log +SETTINGS max_insert_block_size = 200000 +SELECT + 209, + CAST('2022-08-09', 'Date') + toIntervalDay(intDiv(number, 10000)) +FROM numbers(130000) +SETTINGS max_insert_block_size = 200000; + +EXPLAIN ESTIMATE +SELECT count() +FROM url_na_log +PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') +SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, allow_experimental_analyzer=0, parallel_replicas_min_number_of_rows_per_replica=10000; + From a2bdb054ca5a9c205f5d0442313d3c9f4d22cda7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Mar 2024 17:31:37 +0000 Subject: [PATCH 33/66] Cleanup --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +--- .../queries/0_stateless/03006_parallel_replicas_prewhere.sql | 4 +++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 80e2d5afef7..987bf560add 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2111,8 +2111,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. - analysis.prewhere_info - = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); + analysis.prewhere_info = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); analysis.prewhere_info->prewhere_actions->projectInput(false); analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; analysis.prewhere_info->need_filter = true; @@ -2121,7 +2120,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis } else { - chassert(analysis.filter_info->actions); /// Add row level security actions to prewhere. analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index 819526b5596..4b84646c034 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -25,5 +25,7 @@ EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, allow_experimental_analyzer=0, parallel_replicas_min_number_of_rows_per_replica=10000; +SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; +DROP POLICY url_na_log_policy0 ON url_na_log; +DROP TABLE url_na_log; From b9d6f4b3ed999e217d0be22f1580b4d19de4e3d9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 7 Mar 2024 17:42:50 +0000 Subject: [PATCH 34/66] fix deadlock in async inserts via native protocol --- src/Server/TCPHandler.cpp | 4 +++- .../03006_async_insert_deadlock_log.reference | 1 + .../0_stateless/03006_async_insert_deadlock_log.sh | 12 ++++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03006_async_insert_deadlock_log.reference create mode 100755 tests/queries/0_stateless/03006_async_insert_deadlock_log.sh diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d883029408c..a3cdec6a4f3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -936,6 +936,8 @@ void TCPHandler::processInsertQuery() auto result = processAsyncInsertQuery(*insert_queue); if (result.status == AsynchronousInsertQueue::PushResult::OK) { + /// Reset pipeline because it may hold write lock for some storages. + state.io.pipeline.reset(); if (settings.wait_for_async_insert) { size_t timeout_ms = settings.wait_for_async_insert_timeout.totalMilliseconds(); @@ -968,7 +970,7 @@ void TCPHandler::processInsertQuery() else { PushingPipelineExecutor executor(state.io.pipeline); - run_executor(executor, processed_block); + run_executor(executor, std::move(processed_block)); } sendInsertProfileEvents(); diff --git a/tests/queries/0_stateless/03006_async_insert_deadlock_log.reference b/tests/queries/0_stateless/03006_async_insert_deadlock_log.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03006_async_insert_deadlock_log.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03006_async_insert_deadlock_log.sh b/tests/queries/0_stateless/03006_async_insert_deadlock_log.sh new file mode 100755 index 00000000000..f01c34cdbda --- /dev/null +++ b/tests/queries/0_stateless/03006_async_insert_deadlock_log.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "CREATE TABLE t_async_insert_deadlock (a UInt64) ENGINE = Log" + +echo '{"a": 1}' | $CLICKHOUSE_CLIENT --async_insert 1 --wait_for_async_insert 1 --query "INSERT INTO t_async_insert_deadlock FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM t_async_insert_deadlock ORDER BY a" +$CLICKHOUSE_CLIENT --query "DROP TABLE t_async_insert_deadlock" From 5bfb6f08a8cad877950c8776a766e67a13e4f3e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B1=AA=E8=82=A5=E8=82=A5?= Date: Fri, 8 Mar 2024 06:29:15 +0800 Subject: [PATCH 35/66] [Docs] correction arguments --- docs/en/operations/utilities/clickhouse-local.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 437a5f0fff0..93a3fecf3c6 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -201,12 +201,12 @@ Arguments: - `-S`, `--structure` — table structure for input data. - `--input-format` — input format, `TSV` by default. -- `-f`, `--file` — path to data, `stdin` by default. +- `-F`, `--file` — path to data, `stdin` by default. - `-q`, `--query` — queries to execute with `;` as delimiter. `--query` can be specified multiple times, e.g. `--query "SELECT 1" --query "SELECT 2"`. Cannot be used simultaneously with `--queries-file`. - `--queries-file` - file path with queries to execute. `--queries-file` can be specified multiple times, e.g. `--query queries1.sql --query queries2.sql`. Cannot be used simultaneously with `--query`. - `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `-N`, `--table` — table name where to put output data, `table` by default. -- `--format`, `--output-format` — output format, `TSV` by default. +- `-f`, `--format`, `--output-format` — output format, `TSV` by default. - `-d`, `--database` — default database, `_local` by default. - `--stacktrace` — whether to dump debug output in case of exception. - `--echo` — print query before execution. From 582a427931336c02ed1971f5c5daf369b6cd49d3 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Fri, 8 Mar 2024 13:54:37 +0800 Subject: [PATCH 36/66] Add comment for move connection drain from prepare() to work() --- src/Processors/Sources/RemoteSource.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index c09657ece35..17fdac43a75 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -90,6 +90,9 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { + /// Connection drain is a heavy operation that may take a long time. + /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. + /// See issue: https://github.com/ClickHouse/ClickHouse/issues/60844 if (need_drain) { query_executor->finish(); From efb72ca8f1a9d20b2053ce280a22e10039d57ed1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 8 Mar 2024 08:41:08 +0100 Subject: [PATCH 37/66] Add tests for fixed issues --- .../0_stateless/03006_buffer_overflow_join.reference | 0 tests/queries/0_stateless/03006_buffer_overflow_join.sql | 6 ++++++ .../03007_column_nullable_uninitialzed_value.reference | 3 +++ .../03007_column_nullable_uninitialzed_value.sql | 1 + 4 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03006_buffer_overflow_join.reference create mode 100644 tests/queries/0_stateless/03006_buffer_overflow_join.sql create mode 100644 tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.reference create mode 100644 tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql diff --git a/tests/queries/0_stateless/03006_buffer_overflow_join.reference b/tests/queries/0_stateless/03006_buffer_overflow_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03006_buffer_overflow_join.sql b/tests/queries/0_stateless/03006_buffer_overflow_join.sql new file mode 100644 index 00000000000..8c1fa3cecc0 --- /dev/null +++ b/tests/queries/0_stateless/03006_buffer_overflow_join.sql @@ -0,0 +1,6 @@ +CREATE TABLE 03006_buffer_overflow_l (`a` String, `b` Tuple(String, String)) ENGINE = Memory; +INSERT INTO 03006_buffer_overflow_l SELECT * FROM generateRandom() limit 1000; +CREATE TABLE 03006_buffer_overflow_r (`a` LowCardinality(Nullable(String)), `c` Tuple(LowCardinality(String), LowCardinality(String))) ENGINE = Memory; +INSERT INTO 03006_buffer_overflow_r SELECT * FROM generateRandom() limit 1000; + +SELECT a FROM 03006_buffer_overflow_l RIGHT JOIN 03006_buffer_overflow_r USING (a) ORDER BY a ASC NULLS FIRST FORMAT Null; diff --git a/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.reference b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.reference new file mode 100644 index 00000000000..bead7ee1474 --- /dev/null +++ b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.reference @@ -0,0 +1,3 @@ +\N 1000 + +\N 1000 diff --git a/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql new file mode 100644 index 00000000000..9479044e0e0 --- /dev/null +++ b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql @@ -0,0 +1 @@ +SELECT count(NULL) IGNORE NULLS > avg(toDecimal32(NULL)) IGNORE NULLS, count() FROM numbers(1000) WITH TOTALS SETTINGS allow_experimental_analyzer = 1; From b1dcf3183be1c140051a8f6bbd3a541bb4d8d47c Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Fri, 8 Mar 2024 17:37:03 +0800 Subject: [PATCH 38/66] Update settings.md --- docs/zh/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 307159aa5a1..c3b4194ed44 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -1087,8 +1087,8 @@ ClickHouse生成异常 - [isNull](../../sql-reference/operators/index.md#operator-is-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 - [isNotNull](../../sql-reference/operators/index.md#is-not-null) 读取 [null](../../sql-reference/data-types/nullable. md/#finding-null) 子列。 - [count](../../sql-reference/aggregate-functions/reference/count.md) 读取 [null](../../sql-reference/data-types/nullable.md/#finding-null) 子列。 -- [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) 读取 [keys](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 -- [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) 读取 [values](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 +- [mapKeys](../../sql-reference/functions/tuple-map-functions.mdx/#mapkeys) 读取 [keys](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 +- [mapValues](../../sql-reference/functions/tuple-map-functions.mdx/#mapvalues) 读取 [values](../../sql-reference/data-types/map.md/#map-subcolumns) 子列。 可能的值: From 46332f334ccda467abfa673a9e75683a47e24110 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Fri, 8 Mar 2024 15:17:07 +0200 Subject: [PATCH 39/66] rm unused alias set/vector pair --- src/Client/QueryFuzzer.cpp | 16 ---------------- src/Client/QueryFuzzer.h | 4 +--- 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 0597a7c1eed..ea2e5e17afd 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1227,12 +1227,6 @@ void QueryFuzzer::collectFuzzInfoMain(ASTPtr ast) { collectFuzzInfoRecurse(ast); - aliases.clear(); - for (const auto & alias : aliases_set) - { - aliases.push_back(alias); - } - column_like.clear(); for (const auto & [name, value] : column_like_map) { @@ -1285,16 +1279,6 @@ void QueryFuzzer::addColumnLike(ASTPtr ast) void QueryFuzzer::collectFuzzInfoRecurse(ASTPtr ast) { - if (auto * impl = dynamic_cast(ast.get())) - { - if (aliases_set.size() > 1000) - { - aliases_set.clear(); - } - - aliases_set.insert(impl->alias); - } - if (typeid_cast(ast.get())) { addColumnLike(ast); diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index f5465626d96..3bc7b0842d3 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -50,9 +50,7 @@ struct QueryFuzzer // we are currently fuzzing. We add some part from each new query we are asked // to fuzz, and keep this state between queries, so the fuzzing output becomes // more interesting over time, as the queries mix. - std::unordered_set aliases_set; - std::vector aliases; - + // The maps are used for collection, and the vectors are used for random access. std::unordered_map column_like_map; std::vector column_like; From f73a8f2eae72ae37b9d1b5b89fc3bca217536b70 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 8 Mar 2024 18:05:56 +0100 Subject: [PATCH 40/66] More comments for column for system tables (#59016) --- src/Access/Common/QuotaDefs.cpp | 90 ++++++++++++-- src/Access/Common/QuotaDefs.h | 2 + src/Access/Common/RowPolicyDefs.cpp | 9 +- src/Access/Common/RowPolicyDefs.h | 1 + src/Interpreters/AsynchronousInsertLog.cpp | 36 +++--- src/Interpreters/BackupLog.cpp | 36 +++--- src/Interpreters/BlobStorageLog.cpp | 26 ++-- src/Interpreters/CrashLog.cpp | 24 ++-- src/Interpreters/FilesystemCacheLog.cpp | 28 ++--- src/Interpreters/OpenTelemetrySpanLog.cpp | 25 ++-- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/ProcessorsProfileLog.cpp | 36 +++--- src/Interpreters/QueryLog.cpp | 6 +- src/Interpreters/QueryThreadLog.cpp | 94 +++++++-------- src/Interpreters/QueryViewsLog.cpp | 46 +++---- src/Interpreters/S3QueueLog.cpp | 26 ++-- src/Interpreters/SessionLog.cpp | 47 ++++---- src/Interpreters/TraceLog.cpp | 35 +++--- src/Interpreters/TransactionsInfoLog.cpp | 26 ++-- src/Interpreters/ZooKeeperLog.cpp | 66 +++++----- .../System/StorageSystemCertificates.cpp | 21 ++-- src/Storages/System/StorageSystemColumns.cpp | 47 ++++---- .../StorageSystemDataSkippingIndices.cpp | 20 +-- .../System/StorageSystemDetachedParts.cpp | 24 ++-- .../System/StorageSystemDictionaries.cpp | 2 +- src/Storages/System/StorageSystemDisks.cpp | 26 ++-- .../System/StorageSystemFilesystemCache.cpp | 30 ++--- src/Storages/System/StorageSystemGraphite.cpp | 6 +- .../System/StorageSystemKafkaConsumers.cpp | 4 +- src/Storages/System/StorageSystemMerges.cpp | 10 +- src/Storages/System/StorageSystemModels.cpp | 9 +- .../System/StorageSystemMutations.cpp | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 2 + src/Storages/System/StorageSystemOne.cpp | 2 + .../StorageSystemPartMovesBetweenShards.cpp | 29 +++-- src/Storages/System/StorageSystemParts.cpp | 40 +++--- .../System/StorageSystemPartsColumns.cpp | 101 ++++++++-------- .../System/StorageSystemPrivileges.cpp | 22 ++-- .../System/StorageSystemProcesses.cpp | 58 ++++----- .../System/StorageSystemProjectionParts.cpp | 114 +++++++++--------- .../StorageSystemProjectionPartsColumns.cpp | 96 +++++++-------- .../System/StorageSystemQuotaLimits.cpp | 3 +- .../System/StorageSystemQuotaUsage.cpp | 4 +- .../System/StorageSystemRemoteDataPaths.cpp | 16 +-- src/Storages/System/StorageSystemReplicas.cpp | 76 ++++++------ .../System/StorageSystemRowPolicies.cpp | 4 +- src/Storages/System/StorageSystemS3Queue.cpp | 16 +-- .../System/StorageSystemSettingsChanges.cpp | 4 +- .../StorageSystemSettingsProfileElements.cpp | 2 +- .../System/StorageSystemStackTrace.cpp | 10 +- .../System/StorageSystemStoragePolicies.cpp | 20 +-- src/Storages/System/StorageSystemSymbols.cpp | 6 +- src/Storages/System/StorageSystemTables.cpp | 6 +- .../System/StorageSystemTransactions.cpp | 11 +- .../System/StorageSystemUserDirectories.cpp | 9 +- .../System/StorageSystemUserProcesses.cpp | 8 +- src/Storages/System/StorageSystemUsers.cpp | 8 +- .../System/StorageSystemViewRefreshes.cpp | 16 +-- .../02294_system_certificates.reference | 20 +-- ...26_settings_changes_system_table.reference | 4 +- ..._all_columns_should_have_comment.reference | 0 .../02992_all_columns_should_have_comment.sql | 4 + 62 files changed, 845 insertions(+), 728 deletions(-) create mode 100644 tests/queries/0_stateless/02992_all_columns_should_have_comment.reference create mode 100644 tests/queries/0_stateless/02992_all_columns_should_have_comment.sql diff --git a/src/Access/Common/QuotaDefs.cpp b/src/Access/Common/QuotaDefs.cpp index 0e9a4d5a365..04c16a562d2 100644 --- a/src/Access/Common/QuotaDefs.cpp +++ b/src/Access/Common/QuotaDefs.cpp @@ -49,71 +49,135 @@ String QuotaTypeInfo::valueToStringWithName(QuotaValue value) const const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type) { - static constexpr auto make_info = [](const char * raw_name_, UInt64 output_denominator_) + static constexpr auto make_info = [](const char * raw_name_, String current_usage_description_, String max_allowed_usage_description_, UInt64 output_denominator_) { String init_name = raw_name_; boost::to_lower(init_name); String init_keyword = raw_name_; boost::replace_all(init_keyword, "_", " "); bool init_output_as_float = (output_denominator_ != 1); - return QuotaTypeInfo{raw_name_, std::move(init_name), std::move(init_keyword), init_output_as_float, output_denominator_}; + return QuotaTypeInfo + { + .raw_name = raw_name_, + .name = std::move(init_name), + .keyword = std::move(init_keyword), + .current_usage_description = std::move(current_usage_description_), + .max_allowed_usage_description = std::move(max_allowed_usage_description_), + .output_as_float = init_output_as_float, + .output_denominator = output_denominator_ + }; }; switch (type) { case QuotaType::QUERIES: { - static const auto info = make_info("QUERIES", 1); + static const auto info = make_info( + "QUERIES", + "The current number of executed queries.", + "The maximum allowed number of queries of all types allowed to be executed.", + 1 + ); return info; } case QuotaType::QUERY_SELECTS: { - static const auto info = make_info("QUERY_SELECTS", 1); + static const auto info = make_info( + "QUERY_SELECTS", + "The current number of executed SELECT queries.", + "The maximum allowed number of SELECT queries allowed to be executed.", + 1 + ); return info; } case QuotaType::QUERY_INSERTS: { - static const auto info = make_info("QUERY_INSERTS", 1); + static const auto info = make_info( + "QUERY_INSERTS", + "The current number of executed INSERT queries.", + "The maximum allowed number of INSERT queries allowed to be executed.", + 1 + ); return info; } case QuotaType::ERRORS: { - static const auto info = make_info("ERRORS", 1); + static const auto info = make_info( + "ERRORS", + "The current number of queries resulted in an error.", + "The maximum number of queries resulted in an error allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::RESULT_ROWS: { - static const auto info = make_info("RESULT_ROWS", 1); + static const auto info = make_info( + "RESULT_ROWS", + "The current total number of rows in the result set of all queries within the current period of time.", + "The maximum total number of rows in the result set of all queries allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::RESULT_BYTES: { - static const auto info = make_info("RESULT_BYTES", 1); + static const auto info = make_info( + "RESULT_BYTES", + "The current total number of bytes in the result set of all queries within the current period of time.", + "The maximum total number of bytes in the result set of all queries allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::READ_ROWS: { - static const auto info = make_info("READ_ROWS", 1); + static const auto info = make_info( + "READ_ROWS", + "The current total number of rows read during execution of all queries within the current period of time.", + "The maximum number of rows to read during execution of all queries allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::READ_BYTES: { - static const auto info = make_info("READ_BYTES", 1); + static const auto info = make_info( + "READ_BYTES", + "The current total number of bytes read during execution of all queries within the current period of time.", + "The maximum number of bytes to read during execution of all queries allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::EXECUTION_TIME: { - static const auto info = make_info("EXECUTION_TIME", 1000000000 /* execution_time is stored in nanoseconds */); + static const auto info = make_info( + "EXECUTION_TIME", + "The current total amount of time (in nanoseconds) spent to execute queries within the current period of time", + "The maximum amount of time (in nanoseconds) allowed for all queries to execute within the specified period of time", + 1000000000 /* execution_time is stored in nanoseconds */ + ); return info; } case QuotaType::WRITTEN_BYTES: { - static const auto info = make_info("WRITTEN_BYTES", 1); + static const auto info = make_info( + "WRITTEN_BYTES", + "The current total number of bytes written during execution of all queries within the current period of time.", + "The maximum number of bytes to written during execution of all queries allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::FAILED_SEQUENTIAL_AUTHENTICATIONS: { - static const auto info = make_info("FAILED_SEQUENTIAL_AUTHENTICATIONS", 1); + static const auto info = make_info( + "FAILED_SEQUENTIAL_AUtheNTICATIONS", + "The current number of consecutive authentication failures within the current period of time.", + "The maximum number of consecutive authentication failures allowed within the specified period of time.", + 1 + ); return info; } case QuotaType::MAX: break; diff --git a/src/Access/Common/QuotaDefs.h b/src/Access/Common/QuotaDefs.h index 4f849a72b43..6618f01c8f9 100644 --- a/src/Access/Common/QuotaDefs.h +++ b/src/Access/Common/QuotaDefs.h @@ -33,6 +33,8 @@ struct QuotaTypeInfo const char * const raw_name = ""; const String name; /// Lowercased with underscores, e.g. "result_rows". const String keyword; /// Uppercased with spaces, e.g. "RESULT ROWS". + const String current_usage_description; + const String max_allowed_usage_description; const bool output_as_float = false; const UInt64 output_denominator = 1; String valueToString(QuotaValue value) const; diff --git a/src/Access/Common/RowPolicyDefs.cpp b/src/Access/Common/RowPolicyDefs.cpp index b1f882fe971..a9509b6dd76 100644 --- a/src/Access/Common/RowPolicyDefs.cpp +++ b/src/Access/Common/RowPolicyDefs.cpp @@ -33,7 +33,7 @@ String toString(RowPolicyFilterType type) const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType type_) { - static constexpr auto make_info = [](const char * raw_name_) + static constexpr auto make_info = [](const char * raw_name_, const String & comment_) { String init_name = raw_name_; boost::to_lower(init_name); @@ -41,14 +41,17 @@ const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType String init_command = init_name.substr(0, underscore_pos); boost::to_upper(init_command); bool init_is_check = (std::string_view{init_name}.substr(underscore_pos + 1) == "check"); - return RowPolicyFilterTypeInfo{raw_name_, std::move(init_name), std::move(init_command), init_is_check}; + return RowPolicyFilterTypeInfo{raw_name_, std::move(init_name), std::move(init_command), comment_, init_is_check}; }; switch (type_) { case RowPolicyFilterType::SELECT_FILTER: { - static const auto info = make_info("SELECT_FILTER"); + static const auto info = make_info( + "SELECT_FILTER", + "Expression which is used for filtering in SELECT queries." + ); return info; } #if 0 /// Row-level security for INSERT, UPDATE, DELETE is not implemented yet. diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h index 7ffc99e1272..bf2f632e98b 100644 --- a/src/Access/Common/RowPolicyDefs.h +++ b/src/Access/Common/RowPolicyDefs.h @@ -52,6 +52,7 @@ struct RowPolicyFilterTypeInfo const char * const raw_name; const String name; /// Lowercased with underscores, e.g. "select_filter". const String command; /// Uppercased without last word, e.g. "SELECT". + const String description; const bool is_check; /// E.g. false for SELECT_FILTER. static const RowPolicyFilterTypeInfo & get(RowPolicyFilterType type); }; diff --git a/src/Interpreters/AsynchronousInsertLog.cpp b/src/Interpreters/AsynchronousInsertLog.cpp index 5d851f6b47d..0fc39c77fb4 100644 --- a/src/Interpreters/AsynchronousInsertLog.cpp +++ b/src/Interpreters/AsynchronousInsertLog.cpp @@ -33,26 +33,26 @@ ColumnsDescription AsynchronousInsertLogElement::getColumnsDescription() }); return ColumnsDescription{ - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "The date when the async insert happened."}, + {"event_time", std::make_shared(), "The date and time when the async insert finished execution."}, + {"event_time_microseconds", std::make_shared(6), "The date and time when the async insert finished execution with microseconds precision."}, - {"query", std::make_shared()}, - {"database", std::make_shared(std::make_shared())}, - {"table", std::make_shared(std::make_shared())}, - {"format", std::make_shared(std::make_shared())}, - {"query_id", std::make_shared()}, - {"bytes", std::make_shared()}, - {"rows", std::make_shared()}, - {"exception", std::make_shared()}, - {"status", type_status}, - {"data_kind", type_data_kind}, + {"query", std::make_shared(), "Query string."}, + {"database", std::make_shared(std::make_shared()), "The name of the database the table is in."}, + {"table", std::make_shared(std::make_shared()), "Table name."}, + {"format", std::make_shared(std::make_shared()), "Format name."}, + {"query_id", std::make_shared(), "ID of the initial query."}, + {"bytes", std::make_shared(), "Number of inserted bytes."}, + {"rows", std::make_shared(), "Number of inserted rows."}, + {"exception", std::make_shared(), "Exception message."}, + {"status", type_status, "Status of the view. Values: 'Ok' = 1 — Successful insert, 'ParsingError' = 2 — Exception when parsing the data, 'FlushError' = 3 — Exception when flushing the data"}, + {"data_kind", type_data_kind, "The status of the data. Value: 'Parsed' and 'Preprocessed'."}, - {"flush_time", std::make_shared()}, - {"flush_time_microseconds", std::make_shared(6)}, - {"flush_query_id", std::make_shared()}, - {"timeout_milliseconds", std::make_shared()}, + {"flush_time", std::make_shared(), "The date and time when the flush happened."}, + {"flush_time_microseconds", std::make_shared(6), "The date and time when the flush happened with microseconds precision."}, + {"flush_query_id", std::make_shared(), "ID of the flush query."}, + {"timeout_milliseconds", std::make_shared(), "The adaptive timeout calculated for this entry."}, }; } diff --git a/src/Interpreters/BackupLog.cpp b/src/Interpreters/BackupLog.cpp index d5b69bc0728..af6c7cf6234 100644 --- a/src/Interpreters/BackupLog.cpp +++ b/src/Interpreters/BackupLog.cpp @@ -22,24 +22,24 @@ ColumnsDescription BackupLogElement::getColumnsDescription() { return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, - {"id", std::make_shared()}, - {"name", std::make_shared()}, - {"base_backup_name", std::make_shared()}, - {"query_id", std::make_shared()}, - {"status", std::make_shared(getBackupStatusEnumValues())}, - {"error", std::make_shared()}, - {"start_time", std::make_shared()}, - {"end_time", std::make_shared()}, - {"num_files", std::make_shared()}, - {"total_size", std::make_shared()}, - {"num_entries", std::make_shared()}, - {"uncompressed_size", std::make_shared()}, - {"compressed_size", std::make_shared()}, - {"files_read", std::make_shared()}, - {"bytes_read", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "Date of the entry."}, + {"event_time_microseconds", std::make_shared(6), "Time of the entry with microseconds precision."}, + {"id", std::make_shared(), "Identifier of the backup or restore operation."}, + {"name", std::make_shared(), "Name of the backup storage (the contents of the FROM or TO clause)."}, + {"base_backup_name", std::make_shared(), "The name of base backup in case incremental one."}, + {"query_id", std::make_shared(), "The ID of a query associated with a backup operation."}, + {"status", std::make_shared(getBackupStatusEnumValues()), "Operation status."}, + {"error", std::make_shared(), "Error message of the failed operation (empty string for successful operations)."}, + {"start_time", std::make_shared(), "Start time of the operation."}, + {"end_time", std::make_shared(), "End time of the operation."}, + {"num_files", std::make_shared(), "Number of files stored in the backup."}, + {"total_size", std::make_shared(), "Total size of files stored in the backup."}, + {"num_entries", std::make_shared(), "Number of entries in the backup, i.e. the number of files inside the folder if the backup is stored as a folder, or the number of files inside the archive if the backup is stored as an archive. It is not the same as num_files if it's an incremental backup or if it contains empty files or duplicates. The following is always true: num_entries <= num_files."}, + {"uncompressed_size", std::make_shared(), "Uncompressed size of the backup."}, + {"compressed_size", std::make_shared(), "Compressed size of the backup. If the backup is not stored as an archive it equals to uncompressed_size."}, + {"files_read", std::make_shared(), "Number of files read during the restore operation."}, + {"bytes_read", std::make_shared(), "Total size of files read during the restore operation."}, }; } diff --git a/src/Interpreters/BlobStorageLog.cpp b/src/Interpreters/BlobStorageLog.cpp index 520405374ca..f9d5b0d6790 100644 --- a/src/Interpreters/BlobStorageLog.cpp +++ b/src/Interpreters/BlobStorageLog.cpp @@ -26,23 +26,23 @@ ColumnsDescription BlobStorageLogElement::getColumnsDescription() return ColumnsDescription { - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, + {"event_date", std::make_shared(), "Date of the event."}, + {"event_time", std::make_shared(), "Time of the event."}, + {"event_time_microseconds", std::make_shared(6), "Time of the event with microseconds precision."}, - {"event_type", event_enum_type}, + {"event_type", event_enum_type, "Type of the event. Possible values: 'Upload', 'Delete', 'MultiPartUploadCreate', 'MultiPartUploadWrite', 'MultiPartUploadComplete', 'MultiPartUploadAbort'"}, - {"query_id", std::make_shared()}, - {"thread_id", std::make_shared()}, - {"thread_name", std::make_shared()}, + {"query_id", std::make_shared(), "Identifier of the query associated with the event, if any."}, + {"thread_id", std::make_shared(), "Identifier of the thread performing the operation."}, + {"thread_name", std::make_shared(), "Name of the thread performing the operation."}, - {"disk_name", std::make_shared(std::make_shared())}, - {"bucket", std::make_shared()}, - {"remote_path", std::make_shared()}, - {"local_path", std::make_shared()}, - {"data_size", std::make_shared()}, + {"disk_name", std::make_shared(std::make_shared()), "Name of the associated disk."}, + {"bucket", std::make_shared(), "Name of the bucket."}, + {"remote_path", std::make_shared(), "Path to the remote resource."}, + {"local_path", std::make_shared(), "Path to the metadata file on the local system, which references the remote resource."}, + {"data_size", std::make_shared(), "Size of the data involved in the upload event."}, - {"error", std::make_shared()}, + {"error", std::make_shared(), "Error message associated with the event, if any."}, }; } diff --git a/src/Interpreters/CrashLog.cpp b/src/Interpreters/CrashLog.cpp index 4fb81e4bcf7..410ea922429 100644 --- a/src/Interpreters/CrashLog.cpp +++ b/src/Interpreters/CrashLog.cpp @@ -23,18 +23,18 @@ ColumnsDescription CrashLogElement::getColumnsDescription() { return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"timestamp_ns", std::make_shared()}, - {"signal", std::make_shared()}, - {"thread_id", std::make_shared()}, - {"query_id", std::make_shared()}, - {"trace", std::make_shared(std::make_shared())}, - {"trace_full", std::make_shared(std::make_shared())}, - {"version", std::make_shared()}, - {"revision", std::make_shared()}, - {"build_id", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "The hostname where the crash occurred."}, + {"event_date", std::make_shared(), "The date of the crash."}, + {"event_time", std::make_shared(), "The time of the crash."}, + {"timestamp_ns", std::make_shared(), "Timestamp of the event with nanoseconds."}, + {"signal", std::make_shared(), "Signal number."}, + {"thread_id", std::make_shared(), "Thread ID."}, + {"query_id", std::make_shared(), "Query ID."}, + {"trace", std::make_shared(std::make_shared()), "Stack trace at the moment of crash. Each element is a virtual memory address inside ClickHouse server process."}, + {"trace_full", std::make_shared(std::make_shared()), "Stack trace at the moment of crash. Each element contains a called method inside ClickHouse server process."}, + {"version", std::make_shared(), "ClickHouse server version."}, + {"revision", std::make_shared(), "ClickHouse server revision."}, + {"build_id", std::make_shared(), "BuildID that is generated by compiler."}, }; } diff --git a/src/Interpreters/FilesystemCacheLog.cpp b/src/Interpreters/FilesystemCacheLog.cpp index ccfee49a66f..80fe1c3a8ef 100644 --- a/src/Interpreters/FilesystemCacheLog.cpp +++ b/src/Interpreters/FilesystemCacheLog.cpp @@ -38,20 +38,20 @@ ColumnsDescription FilesystemCacheLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"query_id", std::make_shared()}, - {"source_file_path", std::make_shared()}, - {"file_segment_range", std::make_shared(types)}, - {"total_requested_range", std::make_shared(types)}, - {"key", std::make_shared()}, - {"offset", std::make_shared()}, - {"size", std::make_shared()}, - {"read_type", std::make_shared()}, - {"read_from_cache_attempted", std::make_shared()}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, - {"read_buffer_id", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname"}, + {"event_date", std::make_shared(), "Event date"}, + {"event_time", std::make_shared(), "Event time"}, + {"query_id", std::make_shared(), "Id of the query"}, + {"source_file_path", std::make_shared(), "File segment path on filesystem"}, + {"file_segment_range", std::make_shared(types), "File segment range"}, + {"total_requested_range", std::make_shared(types), "Full read range"}, + {"key", std::make_shared(), "File segment key"}, + {"offset", std::make_shared(), "File segment offset"}, + {"size", std::make_shared(), "Read size"}, + {"read_type", std::make_shared(), "Read type: READ_FROM_CACHE, READ_FROM_FS_AND_DOWNLOADED_TO_CACHE, READ_FROM_FS_BYPASSING_CACHE"}, + {"read_from_cache_attempted", std::make_shared(), "Whether reading from cache was attempted"}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "Profile events collected while reading this file segment"}, + {"read_buffer_id", std::make_shared(), "Internal implementation read buffer id"}, }; } diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index fffc1e50da0..aa11749f8a6 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -32,12 +32,17 @@ ColumnsDescription OpenTelemetrySpanLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", low_cardinality_string}, - {"trace_id", std::make_shared()}, - {"span_id", std::make_shared()}, - {"parent_span_id", std::make_shared()}, - {"operation_name", low_cardinality_string}, - {"kind", std::move(span_kind_type)}, + {"hostname", low_cardinality_string, "The hostname where this span was captured."}, + {"trace_id", std::make_shared(), "ID of the trace for executed query."}, + {"span_id", std::make_shared(), "ID of the trace span."}, + {"parent_span_id", std::make_shared(), "ID of the parent trace span."}, + {"operation_name", low_cardinality_string, "The name of the operation."}, + {"kind", std::move(span_kind_type), "The SpanKind of the span. " + "INTERNAL — Indicates that the span represents an internal operation within an application. " + "SERVER — Indicates that the span covers server-side handling of a synchronous RPC or other remote request. " + "CLIENT — Indicates that the span describes a request to some remote service. " + "PRODUCER — Indicates that the span describes the initiators of an asynchronous request. This parent span will often end before the corresponding child CONSUMER span, possibly even before the child span starts. " + "CONSUMER - Indicates that the span describes a child of an asynchronous PRODUCER request."}, // DateTime64 is really unwieldy -- there is no "normal" way to convert // it to an UInt64 count of microseconds, except: // 1) reinterpretAsUInt64(reinterpretAsFixedString(date)), which just @@ -48,10 +53,10 @@ ColumnsDescription OpenTelemetrySpanLogElement::getColumnsDescription() // Also subtraction of two DateTime64 points doesn't work, so you can't // get duration. // It is much less hassle to just use UInt64 of microseconds. - {"start_time_us", std::make_shared()}, - {"finish_time_us", std::make_shared()}, - {"finish_date", std::make_shared()}, - {"attribute", std::make_shared(low_cardinality_string, std::make_shared())}, + {"start_time_us", std::make_shared(), "The start time of the trace span (in microseconds)."}, + {"finish_time_us", std::make_shared(), "The finish time of the trace span (in microseconds)."}, + {"finish_date", std::make_shared(), "The finish date of the trace span."}, + {"attribute", std::make_shared(low_cardinality_string, std::make_shared()), "Attribute depending on the trace span. They are filled in according to the recommendations in the OpenTelemetry standard."}, }; } diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index a7f20a06785..66f933f1afa 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -123,7 +123,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"table_uuid", std::make_shared(), "UUID of the table the data part belongs to."}, {"part_name", std::make_shared(), "Name of the data part."}, {"partition_id", std::make_shared(), "ID of the partition that the data part was inserted to. The column takes the `all` value if the partitioning is by `tuple()`."}, - {"partition", std::make_shared()}, + {"partition", std::make_shared(), "The partition name."}, {"part_type", std::make_shared(), "The type of the part. Possible values: Wide and Compact."}, {"disk_name", std::make_shared(), "The disk name data part lies on."}, {"path_on_disk", std::make_shared(), "Absolute path to the folder with data part files."}, diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index 088d193257c..015b4abc712 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -21,26 +21,26 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription() { return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "The date when the event happened."}, + {"event_time", std::make_shared(), "The date and time when the event happened."}, + {"event_time_microseconds", std::make_shared(6), "The date and time with microseconds precision when the event happened."}, - {"id", std::make_shared()}, - {"parent_ids", std::make_shared(std::make_shared())}, - {"plan_step", std::make_shared()}, - {"plan_group", std::make_shared()}, + {"id", std::make_shared(), "ID of processor."}, + {"parent_ids", std::make_shared(std::make_shared()), "Parent processors IDs."}, + {"plan_step", std::make_shared(), "ID of the query plan step which created this processor. The value is zero if the processor was not added from any step."}, + {"plan_group", std::make_shared(), "Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result."}, - {"initial_query_id", std::make_shared()}, - {"query_id", std::make_shared()}, - {"name", std::make_shared(std::make_shared())}, - {"elapsed_us", std::make_shared()}, - {"input_wait_elapsed_us", std::make_shared()}, - {"output_wait_elapsed_us", std::make_shared()}, - {"input_rows", std::make_shared()}, - {"input_bytes", std::make_shared()}, - {"output_rows", std::make_shared()}, - {"output_bytes", std::make_shared()}, + {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, + {"query_id", std::make_shared(), "ID of the query."}, + {"name", std::make_shared(std::make_shared()), "Name of the processor."}, + {"elapsed_us", std::make_shared(), "Number of microseconds this processor was executed."}, + {"input_wait_elapsed_us", std::make_shared(), "Number of microseconds this processor was waiting for data (from other processor)."}, + {"output_wait_elapsed_us", std::make_shared(), "Number of microseconds this processor was waiting because output port was full."}, + {"input_rows", std::make_shared(), "The number of rows consumed by processor."}, + {"input_bytes", std::make_shared(), "The number of bytes consumed by processor."}, + {"output_rows", std::make_shared(), "The number of rows generated by processor."}, + {"output_bytes", std::make_shared(), "The number of bytes generated by processor."}, }; } diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index ad6e344655b..92f8ddae141 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -134,13 +134,13 @@ ColumnsDescription QueryLogElement::getColumnsDescription() {"used_storages", array_low_cardinality_string, "Canonical names of storages, which were used during query execution."}, {"used_table_functions", array_low_cardinality_string, "Canonical names of table functions, which were used during query execution."}, - {"used_row_policies", array_low_cardinality_string}, + {"used_row_policies", array_low_cardinality_string, "The list of row policies names that were used during query execution."}, - {"transaction_id", getTransactionIDDataType()}, + {"transaction_id", getTransactionIDDataType(), "The identifier of the transaction in scope of which this query was executed."}, {"query_cache_usage", std::move(query_cache_usage_datatype), "Usage of the query cache during query execution. Values: 'Unknown' = Status unknown, 'None' = The query result was neither written into nor read from the query cache, 'Write' = The query result was written into the query cache, 'Read' = The query result was read from the query cache."}, - {"asynchronous_read_counters", std::make_shared(low_cardinality_string, std::make_shared())}, + {"asynchronous_read_counters", std::make_shared(low_cardinality_string, std::make_shared()), "Metrics for asynchronous reading."}, }; } diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index d153e30a4ce..f50458745b9 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -28,58 +28,58 @@ ColumnsDescription QueryThreadLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", low_cardinality_string}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, - {"query_start_time", std::make_shared()}, - {"query_start_time_microseconds", std::make_shared(6)}, - {"query_duration_ms", std::make_shared()}, + {"hostname", low_cardinality_string, "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "The date when the thread has finished execution of the query."}, + {"event_time", std::make_shared(), "The date and time when the thread has finished execution of the query."}, + {"event_time_microseconds", std::make_shared(6), "The date and time when the thread has finished execution of the query with microseconds precision."}, + {"query_start_time", std::make_shared(), "Start time of query execution."}, + {"query_start_time_microseconds", std::make_shared(6), "Start time of query execution with microsecond precision."}, + {"query_duration_ms", std::make_shared(), "Duration of query execution."}, - {"read_rows", std::make_shared()}, - {"read_bytes", std::make_shared()}, - {"written_rows", std::make_shared()}, - {"written_bytes", std::make_shared()}, - {"memory_usage", std::make_shared()}, - {"peak_memory_usage", std::make_shared()}, + {"read_rows", std::make_shared(), "Number of read rows."}, + {"read_bytes", std::make_shared(), "Number of read bytes."}, + {"written_rows", std::make_shared(), "For INSERT queries, the number of written rows. For other queries, the column value is 0."}, + {"written_bytes", std::make_shared(), "For INSERT queries, the number of written bytes. For other queries, the column value is 0."}, + {"memory_usage", std::make_shared(), "The difference between the amount of allocated and freed memory in context of this thread."}, + {"peak_memory_usage", std::make_shared(), "The maximum difference between the amount of allocated and freed memory in context of this thread."}, - {"thread_name", low_cardinality_string}, - {"thread_id", std::make_shared()}, - {"master_thread_id", std::make_shared()}, - {"current_database", low_cardinality_string}, - {"query", std::make_shared()}, - {"normalized_query_hash", std::make_shared()}, + {"thread_name", low_cardinality_string, "Name of the thread."}, + {"thread_id", std::make_shared(), "Internal thread ID."}, + {"master_thread_id", std::make_shared(), "OS initial ID of initial thread."}, + {"current_database", low_cardinality_string, "Name of the current database."}, + {"query", std::make_shared(), "Query string."}, + {"normalized_query_hash", std::make_shared(), "The hash of normalized query - with wiped constanstans, etc."}, - {"is_initial_query", std::make_shared()}, - {"user", low_cardinality_string}, - {"query_id", std::make_shared()}, - {"address", DataTypeFactory::instance().get("IPv6")}, - {"port", std::make_shared()}, - {"initial_user", low_cardinality_string}, - {"initial_query_id", std::make_shared()}, - {"initial_address", DataTypeFactory::instance().get("IPv6")}, - {"initial_port", std::make_shared()}, - {"initial_query_start_time", std::make_shared()}, - {"initial_query_start_time_microseconds", std::make_shared(6)}, - {"interface", std::make_shared()}, - {"is_secure", std::make_shared()}, - {"os_user", low_cardinality_string}, - {"client_hostname", low_cardinality_string}, - {"client_name", low_cardinality_string}, - {"client_revision", std::make_shared()}, - {"client_version_major", std::make_shared()}, - {"client_version_minor", std::make_shared()}, - {"client_version_patch", std::make_shared()}, - {"http_method", std::make_shared()}, - {"http_user_agent", low_cardinality_string}, - {"http_referer", std::make_shared()}, - {"forwarded_for", std::make_shared()}, - {"quota_key", std::make_shared()}, - {"distributed_depth", std::make_shared()}, + {"is_initial_query", std::make_shared(), "Query type. Possible values: 1 — Query was initiated by the client, 0 — Query was initiated by another query for distributed query execution."}, + {"user", low_cardinality_string, "Name of the user who initiated the current query."}, + {"query_id", std::make_shared(), "ID of the query."}, + {"address", DataTypeFactory::instance().get("IPv6"), "IP address that was used to make the query."}, + {"port", std::make_shared(), "The client port that was used to make the query."}, + {"initial_user", low_cardinality_string, "Name of the user who ran the initial query (for distributed query execution)."}, + {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, + {"initial_address", DataTypeFactory::instance().get("IPv6"), "IP address that the parent query was launched from."}, + {"initial_port", std::make_shared(), "The client port that was used to make the parent query."}, + {"initial_query_start_time", std::make_shared(), "Start time of the initial query execution."}, + {"initial_query_start_time_microseconds", std::make_shared(6), "Start time of the initial query execution "}, + {"interface", std::make_shared(), "Interface that the query was initiated from. Possible values: 1 — TCP, 2 — HTTP."}, + {"is_secure", std::make_shared(), "The flag which shows whether the connection was secure."}, + {"os_user", low_cardinality_string, "OSs username who runs clickhouse-client."}, + {"client_hostname", low_cardinality_string, "Hostname of the client machine where the clickhouse-client or another TCP client is run."}, + {"client_name", low_cardinality_string, "The clickhouse-client or another TCP client name."}, + {"client_revision", std::make_shared(), "Revision of the clickhouse-client or another TCP client."}, + {"client_version_major", std::make_shared(), "Major version of the clickhouse-client or another TCP client."}, + {"client_version_minor", std::make_shared(), "Minor version of the clickhouse-client or another TCP client."}, + {"client_version_patch", std::make_shared(), "Patch component of the clickhouse-client or another TCP client version."}, + {"http_method", std::make_shared(), "HTTP method that initiated the query. Possible values: 0 — The query was launched from the TCP interface, 1 — GET method was used., 2 — POST method was used."}, + {"http_user_agent", low_cardinality_string, "The UserAgent header passed in the HTTP request."}, + {"http_referer", std::make_shared(), "HTTP header `Referer` passed in the HTTP query (contains an absolute or partial address of the page making the query)."}, + {"forwarded_for", std::make_shared(), "HTTP header `X-Forwarded-For` passed in the HTTP query."}, + {"quota_key", std::make_shared(), "The 'quota key' specified in the quotas setting."}, + {"distributed_depth", std::make_shared(), "How many times a query was forwarded between servers."}, - {"revision", std::make_shared()}, + {"revision", std::make_shared(), "ClickHouse revision."}, - {"ProfileEvents", std::make_shared(low_cardinality_string, std::make_shared())}, + {"ProfileEvents", std::make_shared(low_cardinality_string, std::make_shared()), "ProfileEvents that measure different metrics for this thread. The description of them could be found in the table system.events."}, }; } diff --git a/src/Interpreters/QueryViewsLog.cpp b/src/Interpreters/QueryViewsLog.cpp index c426f2d3cf0..a5441363340 100644 --- a/src/Interpreters/QueryViewsLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -35,30 +35,34 @@ ColumnsDescription QueryViewsLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, - {"view_duration_ms", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "The date when the last event of the view happened."}, + {"event_time", std::make_shared(), "The date and time when the view finished execution."}, + {"event_time_microseconds", std::make_shared(6), "The date and time when the view finished execution with microseconds precision."}, + {"view_duration_ms", std::make_shared(), "Duration of view execution (sum of its stages) in milliseconds."}, - {"initial_query_id", std::make_shared()}, - {"view_name", std::make_shared()}, - {"view_uuid", std::make_shared()}, - {"view_type", std::move(view_type_datatype)}, - {"view_query", std::make_shared()}, - {"view_target", std::make_shared()}, + {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, + {"view_name", std::make_shared(), "Name of the view."}, + {"view_uuid", std::make_shared(), "UUID of the view."}, + {"view_type", std::move(view_type_datatype), "Type of the view. Values: 'Default' = 1 — Default views. Should not appear in this log, 'Materialized' = 2 — Materialized views, 'Live' = 3 — Live views."}, + {"view_query", std::make_shared(), "The query executed by the view."}, + {"view_target", std::make_shared(), "The name of the view target table."}, - {"read_rows", std::make_shared()}, - {"read_bytes", std::make_shared()}, - {"written_rows", std::make_shared()}, - {"written_bytes", std::make_shared()}, - {"peak_memory_usage", std::make_shared()}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"read_rows", std::make_shared(), "Number of read rows."}, + {"read_bytes", std::make_shared(), "Number of read bytes."}, + {"written_rows", std::make_shared(), "Number of written rows."}, + {"written_bytes", std::make_shared(), "Number of written bytes."}, + {"peak_memory_usage", std::make_shared(), "The maximum difference between the amount of allocated and freed memory in context of this view."}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "ProfileEvents that measure different metrics. The description of them could be found in the table system.events."}, - {"status", std::move(view_status_datatype)}, - {"exception_code", std::make_shared()}, - {"exception", std::make_shared()}, - {"stack_trace", std::make_shared()} + {"status", std::move(view_status_datatype), "Status of the view. Values: " + "'QueryStart' = 1 — Successful start the view execution. Should not appear, " + "'QueryFinish' = 2 — Successful end of the view execution, " + "'ExceptionBeforeStart' = 3 — Exception before the start of the view execution., " + "'ExceptionWhileProcessing' = 4 — Exception during the view execution."}, + {"exception_code", std::make_shared(), "Code of an exception."}, + {"exception", std::make_shared(), "Exception message."}, + {"stack_trace", std::make_shared(), "Stack trace. An empty string, if the query was completed successfully."} }; } diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index 3ed58de0f87..ba990a8ac25 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -25,19 +25,19 @@ ColumnsDescription S3QueueLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"uuid", std::make_shared()}, - {"file_name", std::make_shared()}, - {"rows_processed", std::make_shared()}, - {"status", status_datatype}, - {"processing_start_time", std::make_shared(std::make_shared())}, - {"processing_end_time", std::make_shared(std::make_shared())}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, - {"exception", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname"}, + {"event_date", std::make_shared(), "Event date of writing this log row"}, + {"event_time", std::make_shared(), "Event time of writing this log row"}, + {"database", std::make_shared(), "The name of a database where current S3Queue table lives."}, + {"table", std::make_shared(), "The name of S3Queue table."}, + {"uuid", std::make_shared(), "The UUID of S3Queue table"}, + {"file_name", std::make_shared(), "File name of the processing file"}, + {"rows_processed", std::make_shared(), "Number of processed rows"}, + {"status", status_datatype, "Status of the processing file"}, + {"processing_start_time", std::make_shared(std::make_shared()), "Time of the start of processing the file"}, + {"processing_end_time", std::make_shared(std::make_shared()), "Time of the end of processing the file"}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "Profile events collected while loading this file"}, + {"exception", std::make_shared(), "Exception message if happened"}, }; } diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index a5bc5012292..dc0ac963d0b 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -121,33 +121,36 @@ ColumnsDescription SessionLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", lc_string_datatype}, - {"type", std::move(event_type)}, - {"auth_id", std::make_shared()}, - {"session_id", std::make_shared()}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, + {"hostname", lc_string_datatype, "Hostname of the server executing the query."}, + {"type", std::move(event_type), "Login/logout result. Possible values: " + "LoginFailure — Login error. " + "LoginSuccess — Successful login. " + "Logout — Logout from the system."}, + {"auth_id", std::make_shared(), "Authentication ID, which is a UUID that is automatically generated each time user logins."}, + {"session_id", std::make_shared(), "Session ID that is passed by client via HTTP interface."}, + {"event_date", std::make_shared(), "Login/logout date."}, + {"event_time", std::make_shared(), "Login/logout time."}, + {"event_time_microseconds", std::make_shared(6), "Login/logout starting time with microseconds precision."}, - {"user", std::make_shared(std::make_shared())}, - {"auth_type", std::make_shared(std::move(identified_with_column))}, + {"user", std::make_shared(std::make_shared()), "User name."}, + {"auth_type", std::make_shared(std::move(identified_with_column)), "The authentication type."}, - {"profiles", std::make_shared(lc_string_datatype)}, - {"roles", std::make_shared(lc_string_datatype)}, - {"settings", std::move(settings_type_column)}, + {"profiles", std::make_shared(lc_string_datatype), "The list of profiles set for all roles and/or users."}, + {"roles", std::make_shared(lc_string_datatype), "The list of roles to which the profile is applied."}, + {"settings", std::move(settings_type_column), "Settings that were changed when the client logged in/out."}, - {"client_address", DataTypeFactory::instance().get("IPv6")}, - {"client_port", std::make_shared()}, - {"interface", std::move(interface_type_column)}, + {"client_address", DataTypeFactory::instance().get("IPv6"), "The IP address that was used to log in/out."}, + {"client_port", std::make_shared(), "The client port that was used to log in/out."}, + {"interface", std::move(interface_type_column), "The interface from which the login was initiated."}, - {"client_hostname", std::make_shared()}, - {"client_name", std::make_shared()}, - {"client_revision", std::make_shared()}, - {"client_version_major", std::make_shared()}, - {"client_version_minor", std::make_shared()}, - {"client_version_patch", std::make_shared()}, + {"client_hostname", std::make_shared(), "The hostname of the client machine where the clickhouse-client or another TCP client is run."}, + {"client_name", std::make_shared(), "The clickhouse-client or another TCP client name."}, + {"client_revision", std::make_shared(), "Revision of the clickhouse-client or another TCP client."}, + {"client_version_major", std::make_shared(), "The major version of the clickhouse-client or another TCP client."}, + {"client_version_minor", std::make_shared(), "The minor version of the clickhouse-client or another TCP client."}, + {"client_version_patch", std::make_shared(), "Patch component of the clickhouse-client or another TCP client version."}, - {"failure_reason", std::make_shared()}, + {"failure_reason", std::make_shared(), "The exception message containing the reason for the login/logout failure."}, }; } diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index 26adb0cfc3f..01bedf34f15 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -29,20 +29,27 @@ ColumnsDescription TraceLogElement::getColumnsDescription() { return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared()}, - {"event_time_microseconds", std::make_shared(6)}, - {"timestamp_ns", std::make_shared()}, - {"revision", std::make_shared()}, - {"trace_type", std::make_shared(trace_values)}, - {"thread_id", std::make_shared()}, - {"query_id", std::make_shared()}, - {"trace", std::make_shared(std::make_shared())}, - {"size", std::make_shared()}, - {"ptr", std::make_shared()}, - {"event", std::make_shared(std::make_shared())}, - {"increment", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"event_date", std::make_shared(), "Date of sampling moment."}, + {"event_time", std::make_shared(), "Timestamp of the sampling moment."}, + {"event_time_microseconds", std::make_shared(6), "Timestamp of the sampling moment with microseconds precision."}, + {"timestamp_ns", std::make_shared(), "Timestamp of the sampling moment in nanoseconds."}, + {"revision", std::make_shared(), "ClickHouse server build revision."}, + {"trace_type", std::make_shared(trace_values), "Trace type: " + "`Real` represents collecting stack traces by wall-clock time. " + "`CPU` represents collecting stack traces by CPU time. " + "`Memory` represents collecting allocations and deallocations when memory allocation exceeds the subsequent watermark. " + "`MemorySample` represents collecting random allocations and deallocations. " + "`MemoryPeak` represents collecting updates of peak memory usage. " + "`ProfileEvent` represents collecting of increments of profile events." + }, + {"thread_id", std::make_shared(), "Thread identifier."}, + {"query_id", std::make_shared(), "Query identifier that can be used to get details about a query that was running from the query_log system table."}, + {"trace", std::make_shared(std::make_shared()), "Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process."}, + {"size", std::make_shared(), "For trace types Memory, MemorySample or MemoryPeak is the amount of memory allocated, for other trace types is 0."}, + {"ptr", std::make_shared(), "The address of the allocated chunk."}, + {"event", std::make_shared(std::make_shared()), "For trace type ProfileEvent is the name of updated profile event, for other trace types is an empty string."}, + {"increment", std::make_shared(), "For trace type ProfileEvent is the amount of increment of profile event, for other trace types is 0."}, }; } diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index 4a413439671..d13b31518d2 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -34,22 +34,22 @@ ColumnsDescription TransactionsInfoLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"type", std::move(type_enum)}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared(6)}, - {"thread_id", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "The hostname where transaction was executed."}, + {"type", std::move(type_enum), "The type of the transaction. Possible values: Begin, Commit, Rollback, AddPart, LockPart, UnlockPart."}, + {"event_date", std::make_shared(), "Date of the entry."}, + {"event_time", std::make_shared(6), "Time of the entry"}, + {"thread_id", std::make_shared(), "The identifier of a thread."}, /// which thread? - {"query_id", std::make_shared()}, - {"tid", getTransactionIDDataType()}, - {"tid_hash", std::make_shared()}, + {"query_id", std::make_shared(), "The ID of a query executed in a scope of transaction."}, + {"tid", getTransactionIDDataType(), "The identifier of a transaction."}, + {"tid_hash", std::make_shared(), "The hash of the identifier."}, - {"csn", std::make_shared()}, + {"csn", std::make_shared(), "The Commit Sequence Number"}, - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"uuid", std::make_shared()}, - {"part", std::make_shared()}, + {"database", std::make_shared(), "The name of the database the transaction was executed against."}, + {"table", std::make_shared(), "The name of the table the transaction was executed against."}, + {"uuid", std::make_shared(), "The uuid of the table the transaction was executed against."}, + {"part", std::make_shared(), "The name of the part participated in the transaction."}, // ? }; } diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 9cc31edfe56..6f6d4568064 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -122,49 +122,49 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription() return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared())}, - {"type", std::move(type_enum)}, - {"event_date", std::make_shared()}, - {"event_time", std::make_shared(6)}, - {"thread_id", std::make_shared()}, - {"query_id", std::make_shared()}, - {"address", DataTypeFactory::instance().get("IPv6")}, - {"port", std::make_shared()}, - {"session_id", std::make_shared()}, - {"duration_ms", std::make_shared()}, + {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, + {"type", std::move(type_enum), "Event type in the ZooKeeper client. Can have one of the following values: Request — The request has been sent, Response — The response was received, Finalize — The connection is lost, no response was received."}, + {"event_date", std::make_shared(), "The date when the event happened."}, + {"event_time", std::make_shared(6), "The date and time when the event happened."}, + {"thread_id", std::make_shared(), "The ID of the thread executed this request."}, + {"query_id", std::make_shared(), "The ID of a query in scope of which this request was executed."}, + {"address", DataTypeFactory::instance().get("IPv6"), "IP address of ZooKeeper server that was used to make the request."}, + {"port", std::make_shared(), "The port of ZooKeeper server that was used to make the request."}, + {"session_id", std::make_shared(), "The session ID that the ZooKeeper server sets for each connection."}, + {"duration_ms", std::make_shared(), "The time taken by ZooKeeper to execute the request."}, - {"xid", std::make_shared()}, - {"has_watch", std::make_shared()}, - {"op_num", op_num_enum}, - {"path", std::make_shared()}, + {"xid", std::make_shared(), "The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired response/finalize row."}, + {"has_watch", std::make_shared(), "The request whether the watch has been set."}, + {"op_num", op_num_enum, "The type of request or response."}, + {"path", std::make_shared(), "The path to the ZooKeeper node specified in the request, or an empty string if the request not requires specifying a path."}, - {"data", std::make_shared()}, + {"data", std::make_shared(), "The data written to the ZooKeeper node (for the SET and CREATE requests — what the request wanted to write, for the response to the GET request — what was read) or an empty string."}, - {"is_ephemeral", std::make_shared()}, - {"is_sequential", std::make_shared()}, + {"is_ephemeral", std::make_shared(), "Is the ZooKeeper node being created as an ephemeral."}, + {"is_sequential", std::make_shared(), "Is the ZooKeeper node being created as an sequential."}, - {"version", std::make_shared(std::make_shared())}, + {"version", std::make_shared(std::make_shared()), "The version of the ZooKeeper node that the request expects when executing. This is supported for CHECK, SET, REMOVE requests (is relevant -1 if the request does not check the version or NULL for other requests that do not support version checking)."}, - {"requests_size", std::make_shared()}, - {"request_idx", std::make_shared()}, + {"requests_size", std::make_shared(), "The number of requests included in the multi request (this is a special request that consists of several consecutive ordinary requests and executes them atomically). All requests included in multi request will have the same xid."}, + {"request_idx", std::make_shared(), "The number of the request included in multi request (for multi request — 0, then in order from 1)."}, - {"zxid", std::make_shared()}, - {"error", std::make_shared(error_enum)}, + {"zxid", std::make_shared(), "ZooKeeper transaction ID. The serial number issued by the ZooKeeper server in response to a successfully executed request (0 if the request was not executed/returned an error/the client does not know whether the request was executed)."}, + {"error", std::make_shared(error_enum), "Error code. Can have many values, here are just some of them: ZOK — The request was executed successfully, ZCONNECTIONLOSS — The connection was lost, ZOPERATIONTIMEOUT — The request execution timeout has expired, ZSESSIONEXPIRED — The session has expired, NULL — The request is completed."}, - {"watch_type", std::make_shared(watch_type_enum)}, - {"watch_state", std::make_shared(watch_state_enum)}, + {"watch_type", std::make_shared(watch_type_enum), "The type of the watch event (for responses with op_num = Watch), for the remaining responses: NULL."}, + {"watch_state", std::make_shared(watch_state_enum), "The status of the watch event (for responses with op_num = Watch), for the remaining responses: NULL."}, - {"path_created", std::make_shared()}, + {"path_created", std::make_shared(), "The path to the created ZooKeeper node (for responses to the CREATE request), may differ from the path if the node is created as a sequential."}, - {"stat_czxid", std::make_shared()}, - {"stat_mzxid", std::make_shared()}, - {"stat_pzxid", std::make_shared()}, - {"stat_version", std::make_shared()}, - {"stat_cversion", std::make_shared()}, - {"stat_dataLength", std::make_shared()}, - {"stat_numChildren", std::make_shared()}, + {"stat_czxid", std::make_shared(), "The zxid of the change that caused this ZooKeeper node to be created."}, + {"stat_mzxid", std::make_shared(), "The zxid of the change that last modified this ZooKeeper node."}, + {"stat_pzxid", std::make_shared(), "The transaction ID of the change that last modified children of this ZooKeeper node."}, + {"stat_version", std::make_shared(), "The number of changes to the data of this ZooKeeper node."}, + {"stat_cversion", std::make_shared(), "The number of changes to the children of this ZooKeeper node."}, + {"stat_dataLength", std::make_shared(), "The length of the data field of this ZooKeeper node."}, + {"stat_numChildren", std::make_shared(), "The number of children of this ZooKeeper node."}, - {"children", std::make_shared(std::make_shared())}, + {"children", std::make_shared(std::make_shared()), "The list of child ZooKeeper nodes (for responses to LIST request)."}, }; } diff --git a/src/Storages/System/StorageSystemCertificates.cpp b/src/Storages/System/StorageSystemCertificates.cpp index 0e4c5648b74..5ec683e1784 100644 --- a/src/Storages/System/StorageSystemCertificates.cpp +++ b/src/Storages/System/StorageSystemCertificates.cpp @@ -19,19 +19,18 @@ namespace DB ColumnsDescription StorageSystemCertificates::getColumnsDescription() { - /// TODO: Fill in all the comments. return ColumnsDescription { - {"version", std::make_shared>()}, - {"serial_number", std::make_shared(std::make_shared())}, - {"signature_algo", std::make_shared(std::make_shared())}, - {"issuer", std::make_shared(std::make_shared())}, - {"not_before", std::make_shared(std::make_shared())}, - {"not_after", std::make_shared(std::make_shared())}, - {"subject", std::make_shared(std::make_shared())}, - {"pkey_algo", std::make_shared(std::make_shared())}, - {"path", std::make_shared()}, - {"default", std::make_shared>()} + {"version", std::make_shared>(), "Version of the certificate. Values are 0 for v1, 1 for v2, 2 for v3."}, + {"serial_number", std::make_shared(std::make_shared()), "Serial Number of the certificate assigned by the issuer."}, + {"signature_algo", std::make_shared(std::make_shared()), "Signature Algorithm - an algorithm used by the issuer to sign this certificate."}, + {"issuer", std::make_shared(std::make_shared()), "Issuer - an unique identifier for the Certificate Authority issuing this certificate."}, + {"not_before", std::make_shared(std::make_shared()), "The beginning of the time window when this certificate is valid."}, + {"not_after", std::make_shared(std::make_shared()), "The end of the time window when this certificate is valid."}, + {"subject", std::make_shared(std::make_shared()), "Subject - identifies the owner of the public key."}, + {"pkey_algo", std::make_shared(std::make_shared()), "Public Key Algorithm defines the algorithm the public key can be used with."}, + {"path", std::make_shared(), "Path to the file or directory containing this certificate."}, + {"default", std::make_shared>(), "Certificate is in the default certificate location."} }; } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6bc1208a6a9..5c96c6502af 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -31,27 +31,32 @@ StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_) StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( { - { "database", std::make_shared() }, - { "table", std::make_shared() }, - { "name", std::make_shared() }, - { "type", std::make_shared() }, - { "position", std::make_shared() }, - { "default_kind", std::make_shared() }, - { "default_expression", std::make_shared() }, - { "data_compressed_bytes", std::make_shared() }, - { "data_uncompressed_bytes", std::make_shared() }, - { "marks_bytes", std::make_shared() }, - { "comment", std::make_shared() }, - { "is_in_partition_key", std::make_shared() }, - { "is_in_sorting_key", std::make_shared() }, - { "is_in_primary_key", std::make_shared() }, - { "is_in_sampling_key", std::make_shared() }, - { "compression_codec", std::make_shared() }, - { "character_octet_length", std::make_shared(std::make_shared()) }, - { "numeric_precision", std::make_shared(std::make_shared()) }, - { "numeric_precision_radix", std::make_shared(std::make_shared()) }, - { "numeric_scale", std::make_shared(std::make_shared()) }, - { "datetime_precision", std::make_shared(std::make_shared()) }, + { "database", std::make_shared(), "Database name."}, + { "table", std::make_shared(), "Table name."}, + { "name", std::make_shared(), "Column name."}, + { "type", std::make_shared(), "Column type."}, + { "position", std::make_shared(), "Ordinal position of a column in a table starting with 1."}, + { "default_kind", std::make_shared(), "Expression type (DEFAULT, MATERIALIZED, ALIAS) for the default value, or an empty string if it is not defined."}, + { "default_expression", std::make_shared(), "Expression for the default value, or an empty string if it is not defined."}, + { "data_compressed_bytes", std::make_shared(), "The size of compressed data, in bytes."}, + { "data_uncompressed_bytes", std::make_shared(), "The size of decompressed data, in bytes."}, + { "marks_bytes", std::make_shared(), "The size of marks, in bytes."}, + { "comment", std::make_shared(), "Comment on the column, or an empty string if it is not defined."}, + { "is_in_partition_key", std::make_shared(), "Flag that indicates whether the column is in the partition expression."}, + { "is_in_sorting_key", std::make_shared(), "Flag that indicates whether the column is in the sorting key expression."}, + { "is_in_primary_key", std::make_shared(), "Flag that indicates whether the column is in the primary key expression."}, + { "is_in_sampling_key", std::make_shared(), "Flag that indicates whether the column is in the sampling key expression."}, + { "compression_codec", std::make_shared(), "Compression codec name."}, + { "character_octet_length", std::make_shared(std::make_shared()), + "Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for FixedString data type. Otherwise, the NULL value is returned."}, + { "numeric_precision", std::make_shared(std::make_shared()), + "Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for Decimal types. Otherwise, the NULL value is returned."}, + { "numeric_precision_radix", std::make_shared(std::make_shared()), + "The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for Decimal types. Otherwise, the NULL value is returned."}, + { "numeric_scale", std::make_shared(std::make_shared()), + "The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for Decimal types. Otherwise, the NULL value is returned."}, + { "datetime_precision", std::make_shared(std::make_shared()), + "Decimal precision of DateTime64 data type. For other data types, the NULL value is returned."}, })); setInMemoryMetadata(storage_metadata); diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 2fa74ef23e6..ff782647c79 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -26,16 +26,16 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( { - { "database", std::make_shared() }, - { "table", std::make_shared() }, - { "name", std::make_shared() }, - { "type", std::make_shared() }, - { "type_full", std::make_shared() }, - { "expr", std::make_shared() }, - { "granularity", std::make_shared() }, - { "data_compressed_bytes", std::make_shared() }, - { "data_uncompressed_bytes", std::make_shared() }, - { "marks", std::make_shared()} + { "database", std::make_shared(), "Database name."}, + { "table", std::make_shared(), "Table name."}, + { "name", std::make_shared(), "Index name."}, + { "type", std::make_shared(), "Index type."}, + { "type_full", std::make_shared(), "Index type expression from create statement."}, + { "expr", std::make_shared(), "Expression for the index calculation."}, + { "granularity", std::make_shared(), "The number of granules in the block."}, + { "data_compressed_bytes", std::make_shared(), "The size of compressed data, in bytes."}, + { "data_uncompressed_bytes", std::make_shared(), "The size of decompressed data, in bytes."}, + { "marks", std::make_shared(), "The size of marks, in bytes."} })); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index fa74093a5a5..1eb79744022 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -271,18 +271,18 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"partition_id", std::make_shared(std::make_shared())}, - {"name", std::make_shared()}, - {"bytes_on_disk", std::make_shared()}, - {"modification_time",std::make_shared()}, - {"disk", std::make_shared()}, - {"path", std::make_shared()}, - {"reason", std::make_shared(std::make_shared())}, - {"min_block_number", std::make_shared(std::make_shared())}, - {"max_block_number", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared())}, + {"database", std::make_shared(), "The name of the database this part belongs to."}, + {"table", std::make_shared(), "The name of the table this part belongs to."}, + {"partition_id", std::make_shared(std::make_shared()), "The identifier of the partition this part belongs to."}, + {"name", std::make_shared(), "The name of the part."}, + {"bytes_on_disk", std::make_shared(), "Total size of all the data part files in bytes."}, + {"modification_time",std::make_shared(), "The time the directory with the data part was modified. This usually corresponds to the time when detach happened."}, + {"disk", std::make_shared(), "The name of the disk that stores this data part."}, + {"path", std::make_shared(), "The path of the disk to the file of this data part."}, + {"reason", std::make_shared(std::make_shared()), "The explanation why this part was detached."}, + {"min_block_number", std::make_shared(std::make_shared()), "The minimum number of data parts that make up the current part after merging."}, + {"max_block_number", std::make_shared(std::make_shared()), "The maximum number of data parts that make up the current part after merging."}, + {"level", std::make_shared(std::make_shared()), "Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts."}, }}); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index a19741f92d1..8b528b4a298 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -82,7 +82,7 @@ ColumnsDescription StorageSystemDictionaries::getColumnsDescription() {"attribute.names", std::make_shared(std::make_shared()), "Array of attribute names provided by the dictionary."}, {"attribute.types", std::make_shared(std::make_shared()), "Corresponding array of attribute types provided by the dictionary."}, {"bytes_allocated", std::make_shared(), "Amount of RAM allocated for the dictionary."}, - {"hierarchical_index_bytes_allocated", std::make_shared(), ""}, + {"hierarchical_index_bytes_allocated", std::make_shared(), "Amount of RAM allocated for hierarchical index."}, {"query_count", std::make_shared(), "Number of queries since the dictionary was loaded or since the last successful reboot."}, {"hit_rate", std::make_shared(), "For cache dictionaries, the percentage of uses for which the value was in the cache."}, {"found_rate", std::make_shared(), "The percentage of uses for which the value was found."}, diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 30d64156b22..b010eff2fe3 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -18,19 +18,19 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( { - {"name", std::make_shared()}, - {"path", std::make_shared()}, - {"free_space", std::make_shared()}, - {"total_space", std::make_shared()}, - {"unreserved_space", std::make_shared()}, - {"keep_free_space", std::make_shared()}, - {"type", std::make_shared()}, - {"is_encrypted", std::make_shared()}, - {"is_read_only", std::make_shared()}, - {"is_write_once", std::make_shared()}, - {"is_remote", std::make_shared()}, - {"is_broken", std::make_shared()}, - {"cache_path", std::make_shared()}, + {"name", std::make_shared(), "Name of a disk in the server configuration."}, + {"path", std::make_shared(), "Path to the mount point in the file system."}, + {"free_space", std::make_shared(), "Free space on disk in bytes."}, + {"total_space", std::make_shared(), "Disk volume in bytes."}, + {"unreserved_space", std::make_shared(), "Free space which is not taken by reservations (free_space minus the size of reservations taken by merges, inserts, and other disk write operations currently running)."}, + {"keep_free_space", std::make_shared(), "Amount of disk space that should stay free on disk in bytes. Defined in the keep_free_space_bytes parameter of disk configuration."}, + {"type", std::make_shared(), "The disk type which tells where this disk stores the data - RAM, local drive or remote storage."}, + {"is_encrypted", std::make_shared(), "Flag which shows whether this disk ecrypts the underlying data. "}, + {"is_read_only", std::make_shared(), "Flag which indicates that you can only perform read operations with this disk."}, + {"is_write_once", std::make_shared(), "Flag which indicates if disk is write-once. Which means that it does support BACKUP to this disk, but does not support INSERT into MergeTree table on this disk."}, + {"is_remote", std::make_shared(), "Flag which indicated what operations with this disk involve network interaction."}, + {"is_broken", std::make_shared(), "Flag which indicates if disk is broken. Broken disks will have 0 space and cannot be used."}, + {"cache_path", std::make_shared(), "The path to the cache directory on local drive in case when the disk supports caching."}, })); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 53cd76e4219..cfb388bc232 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -17,21 +17,21 @@ ColumnsDescription StorageSystemFilesystemCache::getColumnsDescription() /// TODO: Fill in all the comments. return ColumnsDescription { - {"cache_name", std::make_shared()}, - {"cache_base_path", std::make_shared()}, - {"cache_path", std::make_shared()}, - {"key", std::make_shared()}, - {"file_segment_range_begin", std::make_shared()}, - {"file_segment_range_end", std::make_shared()}, - {"size", std::make_shared()}, - {"state", std::make_shared()}, - {"cache_hits", std::make_shared()}, - {"references", std::make_shared()}, - {"downloaded_size", std::make_shared()}, - {"kind", std::make_shared()}, - {"unbound", std::make_shared>()}, - {"user_id", std::make_shared()}, - {"file_size", std::make_shared(std::make_shared())}, + {"cache_name", std::make_shared(), "Name of the cache object. Can be used in `SYSTEM DESCRIBE FILESYSTEM CACHE `, `SYSTEM DROP FILESYSTEM CACHE ` commands"}, + {"cache_base_path", std::make_shared(), "Path to the base directory where all caches files (of a cache identidied by `cache_name`) are stored."}, + {"cache_path", std::make_shared(), "Path to a particular cache file, corresponding to a file segment in a source file"}, + {"key", std::make_shared(), "Cache key of the file segment"}, + {"file_segment_range_begin", std::make_shared(), "Offset corresponding to the beginning of the file segment range"}, + {"file_segment_range_end", std::make_shared(), "Offset corresponding to the (including) end of the file segment range"}, + {"size", std::make_shared(), "Size of the file segment"}, + {"state", std::make_shared(), "File segment state (DOWNLOADED, DOWNLOADING, PARTIALLY_DOWNLOADED, ...)"}, + {"cache_hits", std::make_shared(), "Number of cache hits of corresponding file segment"}, + {"references", std::make_shared(), "Number of references to corresponding file segment. Value 1 means that nobody uses it at the moment (the only existing reference is in cache storage itself)"}, + {"downloaded_size", std::make_shared(), "Downloaded size of the file segment"}, + {"kind", std::make_shared(), "File segment kind (used to distringuish between file segments added as a part of 'Temporary data in cache')"}, + {"unbound", std::make_shared>(), "Internal implementation flag"}, + {"user_id", std::make_shared(), "User id of the user which created the file segment"}, + {"file_size", std::make_shared(std::make_shared()), "File size of the file to which current file segment belongs"}, }; } diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index a638a08fac7..eaa386763c2 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -11,7 +11,11 @@ ColumnsDescription StorageSystemGraphite::getColumnsDescription() return ColumnsDescription { {"config_name", std::make_shared(), "graphite_rollup parameter name."}, - {"rule_type", std::make_shared(), ""}, + {"rule_type", std::make_shared(), + "The rule type. Possible values: RuleTypeAll = 0 - default, with regex, compatible with old scheme; " + "RuleTypePlain = 1 - plain metrics, with regex, compatible with old scheme; " + "RuleTypeTagged = 2 - tagged metrics, with regex, compatible with old scheme; " + "RuleTypeTagList = 3 - tagged metrics, with regex (converted to RuleTypeTagged from string like 'retention=10min ; env=(staging|prod)')"}, {"regexp", std::make_shared(), "A pattern for the metric name."}, {"function", std::make_shared(), "The name of the aggregating function."}, {"age", std::make_shared(), "The minimum age of the data in seconds."}, diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 6c9b1681c8b..86713632339 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -41,8 +41,8 @@ ColumnsDescription StorageSystemKafkaConsumers::getColumnsDescription() {"last_rebalance_time", std::make_shared(), "Timestamp of the most recent Kafka rebalance."}, {"num_rebalance_revocations", std::make_shared(), "Number of times the consumer was revoked its partitions."}, {"num_rebalance_assignments", std::make_shared(), "Number of times the consumer was assigned to Kafka cluster."}, - {"is_currently_used", std::make_shared(), "Consumer is in use."}, - {"last_used", std::make_shared(6)}, + {"is_currently_used", std::make_shared(), "The flag which shows whether the consumer is in use."}, + {"last_used", std::make_shared(6), "The last time this consumer was in use."}, {"rdkafka_stat", std::make_shared(), "Library internal statistic. Set statistics_interval_ms to 0 disable, default is 3000 (once in three seconds)."}, }; } diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index fac653e524e..0fca5dc84a2 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -16,12 +16,12 @@ ColumnsDescription StorageSystemMerges::getColumnsDescription() {"elapsed", std::make_shared(), "The time elapsed (in seconds) since the merge started."}, {"progress", std::make_shared(), "The percentage of completed work from 0 to 1."}, {"num_parts", std::make_shared(), "The number of parts to be merged."}, - {"source_part_names", std::make_shared(std::make_shared()), ""}, + {"source_part_names", std::make_shared(std::make_shared()), "The list of source parts names."}, {"result_part_name", std::make_shared(), "The name of the part that will be formed as the result of merging."}, - {"source_part_paths", std::make_shared(std::make_shared()), ""}, - {"result_part_path", std::make_shared(), ""}, - {"partition_id", std::make_shared()}, - {"partition", std::make_shared()}, + {"source_part_paths", std::make_shared(std::make_shared()), "The list of paths for each source part."}, + {"result_part_path", std::make_shared(), "The path of the part that will be formed as the result of merging."}, + {"partition_id", std::make_shared(), "The identifier of the partition where the merge is happening."}, + {"partition", std::make_shared(), "The name of the partition"}, {"is_mutation", std::make_shared(), "1 if this process is a part mutation."}, {"total_size_bytes_compressed", std::make_shared(), "The total size of the compressed data in the merged chunks."}, {"total_size_bytes_uncompressed", std::make_shared(), "The total size of compressed data in the merged chunks."}, diff --git a/src/Storages/System/StorageSystemModels.cpp b/src/Storages/System/StorageSystemModels.cpp index e715238ddd4..8960d0625ba 100644 --- a/src/Storages/System/StorageSystemModels.cpp +++ b/src/Storages/System/StorageSystemModels.cpp @@ -13,13 +13,12 @@ namespace DB ColumnsDescription StorageSystemModels::getColumnsDescription() { - /// TODO: Fill in all the comments. return ColumnsDescription { - { "model_path", std::make_shared() }, - { "type", std::make_shared() }, - { "loading_start_time", std::make_shared() }, - { "loading_duration", std::make_shared() }, + { "model_path", std::make_shared(), "Path to trained model."}, + { "type", std::make_shared(), "Model type. Now catboost only."}, + { "loading_start_time", std::make_shared(), "The time when the loading of the model started."}, + { "loading_duration", std::make_shared(), "How much time did it take to load the model."}, }; } diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 7d263d9468d..60b80e0b0ad 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -38,7 +38,7 @@ ColumnsDescription StorageSystemMutations::getColumnsDescription() "1 if the mutation is completed, " "0 if the mutation is still in process. " }, - { "is_killed", std::make_shared() }, + { "is_killed", std::make_shared(), "Only available in ClickHouse Cloud."}, { "latest_failed_part", std::make_shared(), "The name of the most recent part that could not be mutated."}, { "latest_fail_time", std::make_shared(), "The date and time of the most recent part mutation failure."}, { "latest_fail_reason", std::make_shared(), "The exception message that caused the most recent part mutation failure."}, diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index c5d8b307368..10898f79d10 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -19,6 +19,8 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult : IStorage(table_id), multithreaded(multithreaded_), limit(limit_), offset(offset_) { StorageInMemoryMetadata storage_metadata; + /// This column doesn't have a comment, because otherwise it will be added to all the tables which were created via + /// CREATE TABLE test as numbers(5) storage_metadata.setColumns(ColumnsDescription({{"number", std::make_shared()}})); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 3091ffdb51a..936d55e61a0 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -15,6 +15,8 @@ StorageSystemOne::StorageSystemOne(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; + /// This column doesn't have a comment, because otherwise it will be added to all tables created via: + /// CREATE TABLE test (dummy UInt8) ENGINE = Distributed(`default`, `system.one`) storage_metadata.setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 9dd2ba0b156..9cba92bca12 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -18,28 +18,27 @@ namespace DB ColumnsDescription StorageSystemPartMovesBetweenShards::getColumnsDescription() { - /// TODO: Fill in all the comments return ColumnsDescription { /// Table properties. - { "database", std::make_shared() }, - { "table", std::make_shared() }, + { "database", std::make_shared(), "The name of the database where move is performed."}, + { "table", std::make_shared(), "The name of the table where move is performed."}, /// Constant element properties. - { "task_name", std::make_shared() }, - { "task_uuid", std::make_shared() }, - { "create_time", std::make_shared() }, - { "part_name", std::make_shared() }, - { "part_uuid", std::make_shared() }, - { "to_shard", std::make_shared() }, - { "dst_part_name", std::make_shared() }, + { "task_name", std::make_shared(), "The name of the moving task."}, + { "task_uuid", std::make_shared(), "The identifier of the moving task."}, + { "create_time", std::make_shared(), "The time when the task was created."}, + { "part_name", std::make_shared(), "The name of the part which is in a process of moving."}, + { "part_uuid", std::make_shared(), "The UUID of the part which is in a process of moving."}, + { "to_shard", std::make_shared(), "The name of the destination shard."}, + { "dst_part_name", std::make_shared(), "The result part name."}, /// Processing status of item. - { "update_time", std::make_shared() }, - { "state", std::make_shared() }, - { "rollback", std::make_shared() }, - { "num_tries", std::make_shared() }, - { "last_exception", std::make_shared() }, + { "update_time", std::make_shared(), "The last time update was performed."}, + { "state", std::make_shared(), "The current state of the move."}, + { "rollback", std::make_shared(), "The flag which indicated whether the operation was rolled back."}, + { "num_tries", std::make_shared(), "The number of tries to complete the operation."}, + { "last_exception", std::make_shared(), "The last exception name if any."}, }; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index b6e4ee4161e..1b800fd64a9 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -94,33 +94,33 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"move_ttl_info.min", std::make_shared(std::make_shared()), "Array of date and time values. Each element describes the minimum key value for a TTL MOVE rule."}, {"move_ttl_info.max", std::make_shared(std::make_shared()), "Array of date and time values. Each element describes the maximum key value for a TTL MOVE rule."}, - {"default_compression_codec", std::make_shared()}, + {"default_compression_codec", std::make_shared(), "The name of the codec used to compress this data part (in case when there is no explicit codec for columns)."}, - {"recompression_ttl_info.expression", std::make_shared(std::make_shared())}, - {"recompression_ttl_info.min", std::make_shared(std::make_shared())}, - {"recompression_ttl_info.max", std::make_shared(std::make_shared())}, + {"recompression_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"recompression_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"recompression_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, - {"group_by_ttl_info.expression", std::make_shared(std::make_shared())}, - {"group_by_ttl_info.min", std::make_shared(std::make_shared())}, - {"group_by_ttl_info.max", std::make_shared(std::make_shared())}, + {"group_by_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"group_by_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"group_by_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, - {"rows_where_ttl_info.expression", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.max", std::make_shared(std::make_shared())}, + {"rows_where_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"rows_where_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"rows_where_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, - {"projections", std::make_shared(std::make_shared())}, + {"projections", std::make_shared(std::make_shared()), "The list of projection names calculated for this part."}, - {"visible", std::make_shared()}, - {"creation_tid", getTransactionIDDataType()}, - {"removal_tid_lock", std::make_shared()}, - {"removal_tid", getTransactionIDDataType()}, - {"creation_csn", std::make_shared()}, - {"removal_csn", std::make_shared()}, + {"visible", std::make_shared(), "Flag which indicated whether this part is visible for SELECT queries."}, + {"creation_tid", getTransactionIDDataType(), "ID of transaction that has created/is trying to create this object."}, + {"removal_tid_lock", std::make_shared(), "Hash of removal_tid, used to lock an object for removal."}, + {"removal_tid", getTransactionIDDataType(), "ID of transaction that has removed/is trying to remove this object"}, + {"creation_csn", std::make_shared(), "CSN of transaction that has created this object"}, + {"removal_csn", std::make_shared(), "CSN of transaction that has removed this object"}, - {"has_lightweight_delete", std::make_shared()}, + {"has_lightweight_delete", std::make_shared(), "The flag which indicated whether the part has lightweight delete mask."}, - {"last_removal_attempt_time", std::make_shared()}, - {"removal_state", std::make_shared()}, + {"last_removal_attempt_time", std::make_shared(), "The last time the server tried to delete this part."}, + {"removal_state", std::make_shared(), "The current state of part removal process."}, } ) { diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 833a5e1ec16..f34b0e0cfda 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -22,59 +22,60 @@ namespace DB StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_) : StorageSystemPartsBase(table_id_, ColumnsDescription{ - {"partition", std::make_shared()}, - {"name", std::make_shared()}, - {"uuid", std::make_shared()}, - {"part_type", std::make_shared()}, - {"active", std::make_shared()}, - {"marks", std::make_shared()}, - {"rows", std::make_shared()}, - {"bytes_on_disk", std::make_shared()}, - {"data_compressed_bytes", std::make_shared()}, - {"data_uncompressed_bytes", std::make_shared()}, - {"marks_bytes", std::make_shared()}, - {"modification_time", std::make_shared()}, - {"remove_time", std::make_shared()}, - {"refcount", std::make_shared()}, - {"min_date", std::make_shared()}, - {"max_date", std::make_shared()}, - {"min_time", std::make_shared()}, - {"max_time", std::make_shared()}, - {"partition_id", std::make_shared()}, - {"min_block_number", std::make_shared()}, - {"max_block_number", std::make_shared()}, - {"level", std::make_shared()}, - {"data_version", std::make_shared()}, - {"primary_key_bytes_in_memory", std::make_shared()}, - {"primary_key_bytes_in_memory_allocated", std::make_shared()}, + {"partition", std::make_shared(), "The partition name."}, + {"name", std::make_shared(), "Name of the data part."}, + {"uuid", std::make_shared(), "The parts UUID."}, + {"part_type", std::make_shared(), "The data part storing format. " + "Possible values: Wide — Each column is stored in a separate file in a filesystem, Compact — All columns are stored in one file in a filesystem."}, + {"active", std::make_shared(), "Flag that indicates whether the data part is active. If a data part is active, it’s used in a table. Otherwise, it’s deleted. Inactive data parts remain after merging."}, + {"marks", std::make_shared(), "The number of marks. To get the approximate number of rows in a data part, multiply marks by the index granularity (usually 8192) (this hint does not work for adaptive granularity)."}, + {"rows", std::make_shared(), "The number of rows."}, + {"bytes_on_disk", std::make_shared(), "Total size of all the data part files in bytes."}, + {"data_compressed_bytes", std::make_shared(), "Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"data_uncompressed_bytes", std::make_shared(), "Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"marks_bytes", std::make_shared(),"The size of the file with marks."}, + {"modification_time", std::make_shared(), "The time the directory with the data part was modified. This usually corresponds to the time of data part creation."}, + {"remove_time", std::make_shared(), "The time when the data part became inactive."}, + {"refcount", std::make_shared(), "The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges."}, + {"min_date", std::make_shared(), "The minimum value for the Date column if that is included in the partition key."}, + {"max_date", std::make_shared(), "The maximum value for the Date column if that is included in the partition key."}, + {"min_time", std::make_shared(), "The minimum value for the DateTime column if that is included in the partition key."}, + {"max_time", std::make_shared(), "The maximum value for the DateTime column if that is included in the partition key."}, + {"partition_id", std::make_shared(), "ID of the partition."}, + {"min_block_number", std::make_shared(), "The minimum number of data parts that make up the current part after merging."}, + {"max_block_number", std::make_shared(), "The maximum number of data parts that make up the current part after merging."}, + {"level", std::make_shared(), "Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts."}, + {"data_version", std::make_shared(), "Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than data_version)."}, + {"primary_key_bytes_in_memory", std::make_shared(), "The amount of memory (in bytes) used by primary key values."}, + {"primary_key_bytes_in_memory_allocated", std::make_shared(), "The amount of memory (in bytes) reserved for primary key values."}, - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"engine", std::make_shared()}, - {"disk_name", std::make_shared()}, - {"path", std::make_shared()}, + {"database", std::make_shared(), "Name of the database."}, + {"table", std::make_shared(), "Name of the table."}, + {"engine", std::make_shared(), "Name of the table engine without parameters."}, + {"disk_name", std::make_shared(), "Name of a disk that stores the data part."}, + {"path", std::make_shared(), "Absolute path to the folder with data part files."}, - {"column", std::make_shared()}, - {"type", std::make_shared()}, - {"column_position", std::make_shared()}, - {"default_kind", std::make_shared()}, - {"default_expression", std::make_shared()}, - {"column_bytes_on_disk", std::make_shared()}, - {"column_data_compressed_bytes", std::make_shared()}, - {"column_data_uncompressed_bytes", std::make_shared()}, - {"column_marks_bytes", std::make_shared()}, - {"column_modification_time", std::make_shared(std::make_shared())}, + {"column", std::make_shared(), "Name of the column."}, + {"type", std::make_shared(), "Column type."}, + {"column_position", std::make_shared(), "Ordinal position of a column in a table starting with 1."}, + {"default_kind", std::make_shared(), "Expression type (DEFAULT, MATERIALIZED, ALIAS) for the default value, or an empty string if it is not defined."}, + {"default_expression", std::make_shared(), "Expression for the default value, or an empty string if it is not defined."}, + {"column_bytes_on_disk", std::make_shared(), "Total size of the column in bytes."}, + {"column_data_compressed_bytes", std::make_shared(), "Total size of the compressed data in the column, in bytes."}, + {"column_data_uncompressed_bytes", std::make_shared(), "Total size of the decompressed data in the column, in bytes."}, + {"column_marks_bytes", std::make_shared(), "The size of the marks for column, in bytes."}, + {"column_modification_time", std::make_shared(std::make_shared()), "The last time the column was modified."}, - {"serialization_kind", std::make_shared()}, - {"substreams", std::make_shared(std::make_shared())}, - {"filenames", std::make_shared(std::make_shared())}, - {"subcolumns.names", std::make_shared(std::make_shared())}, - {"subcolumns.types", std::make_shared(std::make_shared())}, - {"subcolumns.serializations", std::make_shared(std::make_shared())}, - {"subcolumns.bytes_on_disk", std::make_shared(std::make_shared())}, - {"subcolumns.data_compressed_bytes", std::make_shared(std::make_shared())}, - {"subcolumns.data_uncompressed_bytes", std::make_shared(std::make_shared())}, - {"subcolumns.marks_bytes", std::make_shared(std::make_shared())}, + {"serialization_kind", std::make_shared(), "Kind of serialization of a column"}, + {"substreams", std::make_shared(std::make_shared()), "Names of substreams to which column is serialized"}, + {"filenames", std::make_shared(std::make_shared()), "Names of files for each substream of a column respectively"}, + {"subcolumns.names", std::make_shared(std::make_shared()), "Names of subcolumns of a column"}, + {"subcolumns.types", std::make_shared(std::make_shared()), "Types of subcolumns of a column"}, + {"subcolumns.serializations", std::make_shared(std::make_shared()), "Kinds of serialization of subcolumns of a column"}, + {"subcolumns.bytes_on_disk", std::make_shared(std::make_shared()), "Sizes in bytes for each subcolumn"}, + {"subcolumns.data_compressed_bytes", std::make_shared(std::make_shared()), "Sizes of the compressed data for each subcolumn, in bytes"}, + {"subcolumns.data_uncompressed_bytes", std::make_shared(std::make_shared()), "Sizes of the decompressed data for each subcolumn, in bytes"}, + {"subcolumns.marks_bytes", std::make_shared(std::make_shared()), "Sizes of the marks for each subcolumn of a column, in bytes"}, } ) { diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 4d749ab0815..58dcf62115e 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -68,13 +68,21 @@ const std::vector> & StorageSystemPrivileges::getAccess ColumnsDescription StorageSystemPrivileges::getColumnsDescription() { - /// TODO: Fill in all the comments. - return ColumnsDescription - { - {"privilege", std::make_shared(getAccessTypeEnumValues())}, - {"aliases", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared(getLevelEnumValues()))}, - {"parent_group", std::make_shared(std::make_shared(getAccessTypeEnumValues()))}, + return ColumnsDescription{ + {"privilege", + std::make_shared(getAccessTypeEnumValues()), + "Name of a privilege which can be used in the GRANT command."}, + {"aliases", + std::make_shared(std::make_shared()), + "List of aliases which can be used instead of the name of the privilege."}, + {"level", + std::make_shared(std::make_shared(getLevelEnumValues())), + "Level of the privilege. GLOBAL privileges can be granted only globally (ON *.*), DATABASE privileges can be granted " + "on a specific database (ON .*) or globally (ON *.*), TABLE privileges can be granted either on a specific table or " + "on a specific database or globally, and COLUMN privileges can be granted like TABLE privileges but also allow to specify columns."}, + {"parent_group", std::make_shared(std::make_shared(getAccessTypeEnumValues())), + "Parent privilege - if the parent privilege is granted then all its children privileges are considered as granted too." + }, }; } diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index b6fd1aabd45..bef98e59687 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -21,35 +21,35 @@ ColumnsDescription StorageSystemProcesses::getColumnsDescription() { auto description = ColumnsDescription { - {"is_initial_query", std::make_shared()}, + {"is_initial_query", std::make_shared(), "Whether this query comes directly from user or was issues by ClickHouse server in a scope of distributed query execution."}, {"user", std::make_shared(), "The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the default user. The field contains the username for a specific query, not for a query that this query initiated."}, {"query_id", std::make_shared(), "Query ID, if defined."}, - {"address", DataTypeFactory::instance().get("IPv6"), "The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at system.processes on the query requestor server."}, - {"port", std::make_shared()}, + {"address", DataTypeFactory::instance().get("IPv6"), "The IP address the query was made from. The same for distributed processing. To track where a distributed query was originally made from, look at system.processes on the query requestor server."}, + {"port", std::make_shared(), "The client port the query was made from."}, - {"initial_user", std::make_shared()}, - {"initial_query_id", std::make_shared()}, - {"initial_address", DataTypeFactory::instance().get("IPv6")}, - {"initial_port", std::make_shared()}, + {"initial_user", std::make_shared(), "Name of the user who ran the initial query (for distributed query execution)."}, + {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, + {"initial_address", DataTypeFactory::instance().get("IPv6"), "IP address that the parent query was launched from."}, + {"initial_port", std::make_shared(), "The client port that was used to make the parent query."}, - {"interface", std::make_shared()}, + {"interface", std::make_shared(), "The interface which was used to send the query. TCP = 1, HTTP = 2, GRPC = 3, MYSQL = 4, POSTGRESQL = 5, LOCAL = 6, TCP_INTERSERVER = 7."}, - {"os_user", std::make_shared()}, - {"client_hostname", std::make_shared()}, - {"client_name", std::make_shared()}, - {"client_revision", std::make_shared()}, - {"client_version_major", std::make_shared()}, - {"client_version_minor", std::make_shared()}, - {"client_version_patch", std::make_shared()}, + {"os_user", std::make_shared(), "Operating system username who runs clickhouse-client."}, + {"client_hostname", std::make_shared(), "Hostname of the client machine where the clickhouse-client or another TCP client is run."}, + {"client_name", std::make_shared(), "The clickhouse-client or another TCP client name."}, + {"client_revision", std::make_shared(), "Revision of the clickhouse-client or another TCP client."}, + {"client_version_major", std::make_shared(), "Major version of the clickhouse-client or another TCP client."}, + {"client_version_minor", std::make_shared(), "Minor version of the clickhouse-client or another TCP client."}, + {"client_version_patch", std::make_shared(), "Patch component of the clickhouse-client or another TCP client version."}, - {"http_method", std::make_shared()}, - {"http_user_agent", std::make_shared()}, - {"http_referer", std::make_shared()}, - {"forwarded_for", std::make_shared()}, + {"http_method", std::make_shared(), "HTTP method that initiated the query. Possible values: 0 — The query was launched from the TCP interface. 1 — GET method was used. 2 — POST method was used."}, + {"http_user_agent", std::make_shared(), "HTTP header UserAgent passed in the HTTP query."}, + {"http_referer", std::make_shared(), "HTTP header Referer passed in the HTTP query (contains an absolute or partial address of the page making the query)."}, + {"forwarded_for", std::make_shared(), "HTTP header X-Forwarded-For passed in the HTTP query."}, - {"quota_key", std::make_shared()}, - {"distributed_depth", std::make_shared()}, + {"quota_key", std::make_shared(), "The quota key specified in the quotas setting (see keyed)."}, + {"distributed_depth", std::make_shared(), "The number of times query was retransmitted between server nodes internally."}, {"elapsed", std::make_shared(), "The time in seconds since request execution started."}, {"is_cancelled", std::make_shared(), "Was query cancelled."}, @@ -57,18 +57,18 @@ ColumnsDescription StorageSystemProcesses::getColumnsDescription() {"read_rows", std::make_shared(), "The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers."}, {"read_bytes", std::make_shared(), "The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers."}, {"total_rows_approx", std::make_shared(), "The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known."}, - {"written_rows", std::make_shared()}, - {"written_bytes", std::make_shared()}, + {"written_rows", std::make_shared(), "The amount of rows written to the storage."}, + {"written_bytes", std::make_shared(), "The amount of bytes written to the storage."}, {"memory_usage", std::make_shared(), "Amount of RAM the query uses. It might not include some types of dedicated memory"}, - {"peak_memory_usage", std::make_shared()}, + {"peak_memory_usage", std::make_shared(), "The current peak of memory usage."}, {"query", std::make_shared(), "The query text. For INSERT, it does not include the data to insert."}, - {"query_kind", std::make_shared()}, + {"query_kind", std::make_shared(), "The type of the query - SELECT, INSERT, etc."}, - {"thread_ids", std::make_shared(std::make_shared())}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, - {"Settings", std::make_shared(std::make_shared(), std::make_shared())}, + {"thread_ids", std::make_shared(std::make_shared()), "The list of identificators of all threads which executed this query."}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "ProfileEvents calculated for this query."}, + {"Settings", std::make_shared(std::make_shared(), std::make_shared()), "The list of modified user-level settings."}, - {"current_database", std::make_shared()}, + {"current_database", std::make_shared(), "The name of the current database."}, }; description.setAliases({ diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 016705f4e66..4bdcea67313 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -19,71 +19,71 @@ namespace DB StorageSystemProjectionParts::StorageSystemProjectionParts(const StorageID & table_id_) : StorageSystemPartsBase(table_id_, ColumnsDescription{ - {"partition", std::make_shared()}, - {"name", std::make_shared()}, - {"part_type", std::make_shared()}, - {"parent_name", std::make_shared()}, - {"parent_uuid", std::make_shared()}, - {"parent_part_type", std::make_shared()}, - {"active", std::make_shared()}, - {"marks", std::make_shared()}, - {"rows", std::make_shared()}, - {"bytes_on_disk", std::make_shared()}, - {"data_compressed_bytes", std::make_shared()}, - {"data_uncompressed_bytes", std::make_shared()}, - {"marks_bytes", std::make_shared()}, - {"parent_marks", std::make_shared()}, - {"parent_rows", std::make_shared()}, - {"parent_bytes_on_disk", std::make_shared()}, - {"parent_data_compressed_bytes", std::make_shared()}, - {"parent_data_uncompressed_bytes", std::make_shared()}, - {"parent_marks_bytes", std::make_shared()}, - {"modification_time", std::make_shared()}, - {"remove_time", std::make_shared()}, - {"refcount", std::make_shared()}, - {"min_date", std::make_shared()}, - {"max_date", std::make_shared()}, - {"min_time", std::make_shared()}, - {"max_time", std::make_shared()}, - {"partition_id", std::make_shared()}, - {"min_block_number", std::make_shared()}, - {"max_block_number", std::make_shared()}, - {"level", std::make_shared()}, - {"data_version", std::make_shared()}, - {"primary_key_bytes_in_memory", std::make_shared()}, - {"primary_key_bytes_in_memory_allocated", std::make_shared()}, - {"is_frozen", std::make_shared()}, + {"partition", std::make_shared(), "The partition name."}, + {"name", std::make_shared(), "Name of the data part."}, + {"part_type", std::make_shared(), "The data part storing format. Possible Values: Wide (a file per column) and Compact (a single file for all columns)."}, + {"parent_name", std::make_shared(), "The name of the source (parent) data part."}, + {"parent_uuid", std::make_shared(), "The UUID of the source (parent) data part."}, + {"parent_part_type", std::make_shared(), "The source (parent) data part storing format."}, + {"active", std::make_shared(), "Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's about to be deleted. Inactive data parts appear after merging and mutating operations."}, + {"marks", std::make_shared(), "The number of marks. To get the approximate number of rows in a data part, multiply marks by the index granularity (usually 8192) (this hint does not work for adaptive granularity)."}, + {"rows", std::make_shared(), "The number of rows."}, + {"bytes_on_disk", std::make_shared(), "Total size of all the data part files in bytes."}, + {"data_compressed_bytes", std::make_shared(), "Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"data_uncompressed_bytes", std::make_shared(), "Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"marks_bytes", std::make_shared(), "The size of the file with marks."}, + {"parent_marks", std::make_shared(), "The number of marks in the source (parent) part."}, + {"parent_rows", std::make_shared(), "The number of rows in the source (parent) part."}, + {"parent_bytes_on_disk", std::make_shared(), "Total size of all the source (parent) data part files in bytes."}, + {"parent_data_compressed_bytes", std::make_shared(), "Total size of compressed data in the source (parent) data part."}, + {"parent_data_uncompressed_bytes", std::make_shared(), "Total size of uncompressed data in the source (parent) data part."}, + {"parent_marks_bytes", std::make_shared(), "The size of the file with marks in the source (parent) data part."}, + {"modification_time", std::make_shared(), "The time the directory with the data part was modified. This usually corresponds to the time of data part creation."}, + {"remove_time", std::make_shared(), "The time when the data part became inactive."}, + {"refcount", std::make_shared(), "The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges."}, + {"min_date", std::make_shared(), "The minimum value of the date key in the data part."}, + {"max_date", std::make_shared(), "The maximum value of the date key in the data part."}, + {"min_time", std::make_shared(), "The minimum value of the date and time key in the data part."}, + {"max_time", std::make_shared(), "The maximum value of the date and time key in the data part."}, + {"partition_id", std::make_shared(), "ID of the partition."}, + {"min_block_number", std::make_shared(), "The minimum number of data parts that make up the current part after merging."}, + {"max_block_number", std::make_shared(), "The maximum number of data parts that make up the current part after merging."}, + {"level", std::make_shared(), "Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts."}, + {"data_version", std::make_shared(), "Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than data_version)."}, + {"primary_key_bytes_in_memory", std::make_shared(), "The amount of memory (in bytes) used by primary key values."}, + {"primary_key_bytes_in_memory_allocated", std::make_shared(), "The amount of memory (in bytes) reserved for primary key values."}, + {"is_frozen", std::make_shared(), "Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. "}, - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"engine", std::make_shared()}, - {"disk_name", std::make_shared()}, - {"path", std::make_shared()}, + {"database", std::make_shared(), "Name of the database."}, + {"table", std::make_shared(), "Name of the table."}, + {"engine", std::make_shared(), "Name of the table engine without parameters."}, + {"disk_name", std::make_shared(), "Name of a disk that stores the data part."}, + {"path", std::make_shared(), "Absolute path to the folder with data part files."}, - {"hash_of_all_files", std::make_shared()}, - {"hash_of_uncompressed_files", std::make_shared()}, - {"uncompressed_hash_of_compressed_files", std::make_shared()}, + {"hash_of_all_files", std::make_shared(), "sipHash128 of compressed files."}, + {"hash_of_uncompressed_files", std::make_shared(), "sipHash128 of uncompressed files (files with marks, index file etc.)."}, + {"uncompressed_hash_of_compressed_files", std::make_shared(), "sipHash128 of data in the compressed files as if they were uncompressed."}, - {"delete_ttl_info_min", std::make_shared()}, - {"delete_ttl_info_max", std::make_shared()}, + {"delete_ttl_info_min", std::make_shared(), "The minimum value of the date and time key for TTL DELETE rule."}, + {"delete_ttl_info_max", std::make_shared(), "The maximum value of the date and time key for TTL DELETE rule."}, - {"move_ttl_info.expression", std::make_shared(std::make_shared())}, - {"move_ttl_info.min", std::make_shared(std::make_shared())}, - {"move_ttl_info.max", std::make_shared(std::make_shared())}, + {"move_ttl_info.expression", std::make_shared(std::make_shared()), "Array of expressions. Each expression defines a TTL MOVE rule."}, + {"move_ttl_info.min", std::make_shared(std::make_shared()), "Array of date and time values. Each element describes the minimum key value for a TTL MOVE rule."}, + {"move_ttl_info.max", std::make_shared(std::make_shared()), "Array of date and time values. Each element describes the maximum key value for a TTL MOVE rule."}, - {"default_compression_codec", std::make_shared()}, + {"default_compression_codec", std::make_shared(), "The name of the codec used to compress this data part (in case when there is no explicit codec for columns)."}, - {"recompression_ttl_info.expression", std::make_shared(std::make_shared())}, - {"recompression_ttl_info.min", std::make_shared(std::make_shared())}, - {"recompression_ttl_info.max", std::make_shared(std::make_shared())}, + {"recompression_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"recompression_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"recompression_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, - {"group_by_ttl_info.expression", std::make_shared(std::make_shared())}, - {"group_by_ttl_info.min", std::make_shared(std::make_shared())}, - {"group_by_ttl_info.max", std::make_shared(std::make_shared())}, + {"group_by_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"group_by_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"group_by_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, - {"rows_where_ttl_info.expression", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.max", std::make_shared(std::make_shared())} + {"rows_where_ttl_info.expression", std::make_shared(std::make_shared()), "The TTL expression."}, + {"rows_where_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression within this part. Used to understand whether we have at least one row with expired TTL."}, + {"rows_where_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression within this part. Used to understand whether we have all rows with expired TTL."}, } ) { diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index 2ff25f86366..9521605688d 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -19,56 +19,56 @@ namespace DB StorageSystemProjectionPartsColumns::StorageSystemProjectionPartsColumns(const StorageID & table_id_) : StorageSystemPartsBase(table_id_, ColumnsDescription{ - {"partition", std::make_shared()}, - {"name", std::make_shared()}, - {"part_type", std::make_shared()}, - {"parent_name", std::make_shared()}, - {"parent_uuid", std::make_shared()}, - {"parent_part_type", std::make_shared()}, - {"active", std::make_shared()}, - {"marks", std::make_shared()}, - {"rows", std::make_shared()}, - {"bytes_on_disk", std::make_shared()}, - {"data_compressed_bytes", std::make_shared()}, - {"data_uncompressed_bytes", std::make_shared()}, - {"marks_bytes", std::make_shared()}, - {"parent_marks", std::make_shared()}, - {"parent_rows", std::make_shared()}, - {"parent_bytes_on_disk", std::make_shared()}, - {"parent_data_compressed_bytes", std::make_shared()}, - {"parent_data_uncompressed_bytes", std::make_shared()}, - {"parent_marks_bytes", std::make_shared()}, - {"modification_time", std::make_shared()}, - {"remove_time", std::make_shared()}, - {"refcount", std::make_shared()}, - {"min_date", std::make_shared()}, - {"max_date", std::make_shared()}, - {"min_time", std::make_shared()}, - {"max_time", std::make_shared()}, - {"partition_id", std::make_shared()}, - {"min_block_number", std::make_shared()}, - {"max_block_number", std::make_shared()}, - {"level", std::make_shared()}, - {"data_version", std::make_shared()}, - {"primary_key_bytes_in_memory", std::make_shared()}, - {"primary_key_bytes_in_memory_allocated", std::make_shared()}, + {"partition", std::make_shared(), "The partition name. "}, + {"name", std::make_shared(), "Name of the data part."}, + {"part_type", std::make_shared(), "The data part storing format."}, + {"parent_name", std::make_shared(), "The name of the source (parent) data part."}, + {"parent_uuid", std::make_shared(), "The UUID of the source (parent) data part."}, + {"parent_part_type", std::make_shared(), "The source (parent) data part storing format."}, + {"active", std::make_shared(), "Flag that indicates whether the data part is active"}, + {"marks", std::make_shared(), "The number of marks."}, + {"rows", std::make_shared(), "The number of rows."}, + {"bytes_on_disk", std::make_shared(), "Total size of all the data part files in bytes."}, + {"data_compressed_bytes", std::make_shared(), "Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"data_uncompressed_bytes", std::make_shared(), "Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included."}, + {"marks_bytes", std::make_shared(), "The size of the file with marks."}, + {"parent_marks", std::make_shared(), "The number of marks in the source (parent) part."}, + {"parent_rows", std::make_shared(), "The number of rows in the source (parent) part."}, + {"parent_bytes_on_disk", std::make_shared(), "Total size of all the source (parent) data part files in bytes."}, + {"parent_data_compressed_bytes", std::make_shared(), "Total size of compressed data in the source (parent) data part."}, + {"parent_data_uncompressed_bytes", std::make_shared(), "Total size of uncompressed data in the source (parent) data part."}, + {"parent_marks_bytes", std::make_shared(), "The size of the file with marks in the source (parent) data part."}, + {"modification_time", std::make_shared(), "The time the directory with the data part was modified. This usually corresponds to the time of data part creation."}, + {"remove_time", std::make_shared(), "The time when the data part became inactive."}, + {"refcount", std::make_shared(), "The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges."}, + {"min_date", std::make_shared(), "The minimum value for the Date column if that is included in the partition key."}, + {"max_date", std::make_shared(), "The maximum value for the Date column if that is included in the partition key."}, + {"min_time", std::make_shared(), "The minimum value for the DateTime column if that is included in the partition key."}, + {"max_time", std::make_shared(), "The maximum value for the DateTime column if that is included in the partition key."}, + {"partition_id", std::make_shared(), "ID of the partition."}, + {"min_block_number", std::make_shared(), "The minimum number of data parts that make up the current part after merging."}, + {"max_block_number", std::make_shared(), "The maximum number of data parts that make up the current part after merging."}, + {"level", std::make_shared(), "Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts."}, + {"data_version", std::make_shared(), "Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than data_version)."}, + {"primary_key_bytes_in_memory", std::make_shared(), "The amount of memory (in bytes) used by primary key values."}, + {"primary_key_bytes_in_memory_allocated", std::make_shared(), "The amount of memory (in bytes) reserved for primary key values."}, - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"engine", std::make_shared()}, - {"disk_name", std::make_shared()}, - {"path", std::make_shared()}, + {"database", std::make_shared(), "Name of the database."}, + {"table", std::make_shared(), "Name of the table."}, + {"engine", std::make_shared(), "Name of the table engine without parameters."}, + {"disk_name", std::make_shared(), "Name of a disk that stores the data part."}, + {"path", std::make_shared(), "Absolute path to the folder with data part files."}, - {"column", std::make_shared()}, - {"type", std::make_shared()}, - {"column_position", std::make_shared()}, - {"default_kind", std::make_shared()}, - {"default_expression", std::make_shared()}, - {"column_bytes_on_disk", std::make_shared()}, - {"column_data_compressed_bytes", std::make_shared()}, - {"column_data_uncompressed_bytes", std::make_shared()}, - {"column_marks_bytes", std::make_shared()}, - {"column_modification_time", std::make_shared(std::make_shared())}, + {"column", std::make_shared(), "Name of the column."}, + {"type", std::make_shared(), "Column type."}, + {"column_position", std::make_shared(), "Ordinal position of a column in a table starting with 1."}, + {"default_kind", std::make_shared(), "Expression type (DEFAULT, MATERIALIZED, ALIAS) for the default value, or an empty string if it is not defined."}, + {"default_expression", std::make_shared(), "Expression for the default value, or an empty string if it is not defined."}, + {"column_bytes_on_disk", std::make_shared(), "Total size of the column in bytes."}, + {"column_data_compressed_bytes", std::make_shared(), "Total size of compressed data in the column, in bytes."}, + {"column_data_uncompressed_bytes", std::make_shared(), "Total size of the decompressed data in the column, in bytes."}, + {"column_marks_bytes", std::make_shared(), "The size of the column with marks, in bytes."}, + {"column_modification_time", std::make_shared(std::make_shared()), "The last time the column was modified."}, } ) { diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index dba449d4f1d..65acfba0c1b 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -68,7 +68,8 @@ ColumnsDescription StorageSystemQuotaLimits::getColumnsDescription() data_type = std::make_shared(); else data_type = std::make_shared(); - result.add({column_name, std::make_shared(data_type)}); + + result.add({column_name, std::make_shared(data_type), type_info.max_allowed_usage_description}); } return result; diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index 2df36aee240..da9c174b0d3 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -81,8 +81,8 @@ ColumnsDescription StorageSystemQuotaUsage::getColumnsDescriptionImpl(bool add_c data_type = std::make_shared(); else data_type = std::make_shared(); - description.add({column_name, std::make_shared(data_type)}); - description.add({String("max_") + column_name, std::make_shared(data_type)}); + description.add({column_name, std::make_shared(data_type), type_info.current_usage_description}); + description.add({String("max_") + column_name, std::make_shared(data_type), type_info.max_allowed_usage_description}); } return description; diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 55002d6d00f..7cc1951fc05 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -20,14 +20,14 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( { - {"disk_name", std::make_shared()}, - {"path", std::make_shared()}, - {"cache_base_path", std::make_shared()}, - {"local_path", std::make_shared()}, - {"remote_path", std::make_shared()}, - {"size", std::make_shared()}, - {"common_prefix_for_blobs", std::make_shared()}, - {"cache_paths", std::make_shared(std::make_shared())}, + {"disk_name", std::make_shared(), "Disk name."}, + {"path", std::make_shared(), "Disk path."}, + {"cache_base_path", std::make_shared(), "Base directory of cache files."}, + {"local_path", std::make_shared(), "Path of ClickHouse file, also used as metadata path."}, + {"remote_path", std::make_shared(), "Blob path in object storage, with which ClickHouse file is associated with."}, + {"size", std::make_shared(), "Size of the file (compressed)."}, + {"common_prefix_for_blobs", std::make_shared(), "Common prefix for blobs in object storage."}, + {"cache_paths", std::make_shared(std::make_shared()), "Cache files for corresponding blob."}, })); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 26dead8cb01..af8d67cbc21 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -196,41 +196,47 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ - { "database", std::make_shared() }, - { "table", std::make_shared() }, - { "engine", std::make_shared() }, - { "is_leader", std::make_shared() }, - { "can_become_leader", std::make_shared() }, - { "is_readonly", std::make_shared() }, - { "is_session_expired", std::make_shared() }, - { "future_parts", std::make_shared() }, - { "parts_to_check", std::make_shared() }, - { "zookeeper_name", std::make_shared() }, - { "zookeeper_path", std::make_shared() }, - { "replica_name", std::make_shared() }, - { "replica_path", std::make_shared() }, - { "columns_version", std::make_shared() }, - { "queue_size", std::make_shared() }, - { "inserts_in_queue", std::make_shared() }, - { "merges_in_queue", std::make_shared() }, - { "part_mutations_in_queue", std::make_shared() }, - { "queue_oldest_time", std::make_shared() }, - { "inserts_oldest_time", std::make_shared() }, - { "merges_oldest_time", std::make_shared() }, - { "part_mutations_oldest_time", std::make_shared() }, - { "oldest_part_to_get", std::make_shared() }, - { "oldest_part_to_merge_to", std::make_shared() }, - { "oldest_part_to_mutate_to", std::make_shared() }, - { "log_max_index", std::make_shared() }, - { "log_pointer", std::make_shared() }, - { "last_queue_update", std::make_shared() }, - { "absolute_delay", std::make_shared() }, - { "total_replicas", std::make_shared() }, - { "active_replicas", std::make_shared() }, - { "lost_part_count", std::make_shared() }, - { "last_queue_update_exception", std::make_shared() }, - { "zookeeper_exception", std::make_shared() }, - { "replica_is_active", std::make_shared(std::make_shared(), std::make_shared()) } + { "database", std::make_shared(), "Database name."}, + { "table", std::make_shared(), "Table name."}, + { "engine", std::make_shared(), "Table engine name."}, + { "is_leader", std::make_shared(), "Whether the replica is the leader. Multiple replicas can be leaders at the same time. " + "A replica can be prevented from becoming a leader using the merge_tree setting replicated_can_become_leader. " + "The leaders are responsible for scheduling background merges. " + "Note that writes can be performed to any replica that is available and has a session in ZK, regardless of whether it is a leader."}, + { "can_become_leader", std::make_shared(), "Whether the replica can be a leader."}, + { "is_readonly", std::make_shared(), "Whether the replica is in read-only mode. This mode is turned on if the config does not have sections with ClickHouse Keeper, " + "if an unknown error occurred when reinitializing sessions in ClickHouse Keeper, and during session reinitialization in ClickHouse Keeper."}, + { "is_session_expired", std::make_shared(), "Whether the session with ClickHouse Keeper has expired. Basically the same as `is_readonly`."}, + { "future_parts", std::make_shared(), "The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet."}, + { "parts_to_check", std::make_shared(), "The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged."}, + { "zookeeper_name", std::make_shared(), "The name of the the [Zoo]Keeper cluster (possibly auxiliary one) where the table's metadata is stored"}, + { "zookeeper_path", std::make_shared(), "Path to table data in ClickHouse Keeper."}, + { "replica_name", std::make_shared(), "Replica name in ClickHouse Keeper. Different replicas of the same table have different names."}, + { "replica_path", std::make_shared(), "Path to replica data in ClickHouse Keeper. The same as concatenating 'zookeeper_path/replicas/replica_path'."}, + { "columns_version", std::make_shared(), "Version number of the table structure. Indicates how many times ALTER was performed. " + "If replicas have different versions, it means some replicas haven't made all of the ALTERs yet."}, + { "queue_size", std::make_shared(), "Size of the queue for operations waiting to be performed. Operations include inserting blocks of data, merges, and certain other actions. It usually coincides with future_parts."}, + { "inserts_in_queue", std::make_shared(), "Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong."}, + { "merges_in_queue", std::make_shared(), "The number of merges waiting to be made. Sometimes merges are lengthy, so this value may be greater than zero for a long time."}, + { "part_mutations_in_queue", std::make_shared(), "The number of mutations waiting to be made."}, + { "queue_oldest_time", std::make_shared(), "If `queue_size` greater than 0, shows when the oldest operation was added to the queue."}, + { "inserts_oldest_time", std::make_shared(), "See `queue_oldest_time`."}, + { "merges_oldest_time", std::make_shared(), "See `queue_oldest_time`."}, + { "part_mutations_oldest_time", std::make_shared(), "See `queue_oldest_time`."}, + { "oldest_part_to_get", std::make_shared(), "The name of the part to fetch from other replicas obtained from the oldest GET_PARTS entry in the replication queue."}, + { "oldest_part_to_merge_to", std::make_shared(), "The result part name to merge to obtained from the oldest MERGE_PARTS entry in the replication queue."}, + { "oldest_part_to_mutate_to", std::make_shared(), "The result part name to mutate to obtained from the oldest MUTATE_PARTS entry in the replication queue."}, + { "log_max_index", std::make_shared(), "Maximum entry number in the log of general activity."}, + { "log_pointer", std::make_shared(), "Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. " + "If log_pointer is much smaller than log_max_index, something is wrong."}, + { "last_queue_update", std::make_shared(), "When the queue was updated last time."}, + { "absolute_delay", std::make_shared(), "How big lag in seconds the current replica has."}, + { "total_replicas", std::make_shared(), "The total number of known replicas of this table."}, + { "active_replicas", std::make_shared(), "The number of replicas of this table that have a session in ClickHouse Keeper (i.e., the number of functioning replicas)."}, + { "lost_part_count", std::make_shared(), "The number of data parts lost in the table by all replicas in total since table creation. Value is persisted in ClickHouse Keeper and can only increase."}, + { "last_queue_update_exception", std::make_shared(), "When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions."}, + { "zookeeper_exception", std::make_shared(), "The last exception message, got if the error happened when fetching the info from ClickHouse Keeper."}, + { "replica_is_active", std::make_shared(std::make_shared(), std::make_shared()), "Map between replica name and is replica active."} })); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index ea819e88993..93c5ba60a7f 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -38,8 +38,8 @@ ColumnsDescription StorageSystemRowPolicies::getColumnsDescription() for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const String & column_name = RowPolicyFilterTypeInfo::get(filter_type).name; - description.add({column_name, std::make_shared(std::make_shared())}); + const auto & filter_type_info = RowPolicyFilterTypeInfo::get(filter_type); + description.add({filter_type_info.name, std::make_shared(std::make_shared()), filter_type_info.description}); } description.add({"is_restrictive", std::make_shared(), diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 557f0fd1208..a6bb7da2b6e 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -25,14 +25,14 @@ ColumnsDescription StorageSystemS3Queue::getColumnsDescription() /// TODO: Fill in all the comments return ColumnsDescription { - {"zookeeper_path", std::make_shared()}, - {"file_name", std::make_shared()}, - {"rows_processed", std::make_shared()}, - {"status", std::make_shared()}, - {"processing_start_time", std::make_shared(std::make_shared())}, - {"processing_end_time", std::make_shared(std::make_shared())}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, - {"exception", std::make_shared()}, + {"zookeeper_path", std::make_shared(), "Path in zookeeper to S3Queue metadata"}, + {"file_name", std::make_shared(), "File name of a file which is being processed by S3Queue"}, + {"rows_processed", std::make_shared(), "Currently processed number of rows"}, + {"status", std::make_shared(), "Status of processing: Processed, Processing, Failed"}, + {"processing_start_time", std::make_shared(std::make_shared()), "Time at which processing of the file started"}, + {"processing_end_time", std::make_shared(std::make_shared()), "Time at which processing of the file ended"}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "Profile events collected during processing of the file"}, + {"exception", std::make_shared(), "Exception which happened during processing"}, }; } diff --git a/src/Storages/System/StorageSystemSettingsChanges.cpp b/src/Storages/System/StorageSystemSettingsChanges.cpp index ecac76b73a3..de47ec52031 100644 --- a/src/Storages/System/StorageSystemSettingsChanges.cpp +++ b/src/Storages/System/StorageSystemSettingsChanges.cpp @@ -12,7 +12,7 @@ ColumnsDescription StorageSystemSettingsChanges::getColumnsDescription() /// TODO: Fill in all the comments return ColumnsDescription { - {"version", std::make_shared()}, + {"version", std::make_shared(), "The ClickHouse server version."}, {"changes", std::make_shared(std::make_shared( DataTypes{ @@ -20,7 +20,7 @@ ColumnsDescription StorageSystemSettingsChanges::getColumnsDescription() std::make_shared(), std::make_shared(), std::make_shared()}, - Names{"name", "previous_value", "new_value", "reason"}))}, + Names{"name", "previous_value", "new_value", "reason"})), "The list of changes in settings which changed the behaviour of ClickHouse."}, }; } diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index 6ac5d13a249..2af3e6dfd05 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -42,7 +42,7 @@ ColumnsDescription StorageSystemSettingsProfileElements::getColumnsDescription() {"value", std::make_shared(std::make_shared()), "Setting value."}, {"min", std::make_shared(std::make_shared()), "The minimum value of the setting. NULL if not set."}, {"max", std::make_shared(std::make_shared()), "The maximum value of the setting. NULL if not set."}, - {"writability", std::make_shared(std::make_shared(getSettingConstraintWritabilityEnumValues()))}, + {"writability", std::make_shared(std::make_shared(getSettingConstraintWritabilityEnumValues())), "The property which shows whether a setting can be changed or not."}, {"inherit_profile", std::make_shared(std::make_shared()), "A parent profile for this setting profile. NULL if not set. " "Setting profile will inherit all the settings' values and constraints (min, max, readonly) from its parent profiles." diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 74864bb50e1..ba7433fb9ae 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -507,11 +507,11 @@ StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ - { "thread_name", std::make_shared() }, - { "thread_id", std::make_shared() }, - { "query_id", std::make_shared() }, - { "trace", std::make_shared(std::make_shared()) }, - }, { /* aliases */ })); + {"thread_name", std::make_shared(), "The name of the thread."}, + {"thread_id", std::make_shared(), "The thread identifier"}, + {"query_id", std::make_shared(), "The ID of the query this thread belongs to."}, + {"trace", std::make_shared(std::make_shared()), "The stacktrace of this thread. Basically just an array of addresses."}, + })); setInMemoryMetadata(storage_metadata); notification_pipe.open(); diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 39e99884e1d..21251136f7d 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -37,16 +37,16 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & tab StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns( ColumnsDescription({ - {"policy_name", std::make_shared()}, - {"volume_name", std::make_shared()}, - {"volume_priority", std::make_shared()}, - {"disks", std::make_shared(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()}, - {"perform_ttl_move_on_insert", std::make_shared()}, - {"load_balancing", std::make_shared(getTypeEnumValues())} + {"policy_name", std::make_shared(), "The name of the storage policy."}, + {"volume_name", std::make_shared(), "The name of the volume."}, + {"volume_priority", std::make_shared(), "The priority of the volume."}, + {"disks", std::make_shared(std::make_shared()), "The list of all disks names which are a part of this storage policy."}, + {"volume_type", std::make_shared(getTypeEnumValues()), "The type of the volume - JBOD or a single disk."}, + {"max_data_part_size", std::make_shared(), "the maximum size of a part that can be stored on any of the volumes disks."}, + {"move_factor", std::make_shared(), "When the amount of available space gets lower than this factor, data automatically starts to move on the next volume if any (by default, 0.1)."}, + {"prefer_not_to_merge", std::make_shared(), "You should not use this setting. Disables merging of data parts on this volume (this is harmful and leads to performance degradation)."}, + {"perform_ttl_move_on_insert", std::make_shared(), "Disables TTL move on data part INSERT. By default (if enabled) 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."}, + {"load_balancing", std::make_shared(getTypeEnumValues()), "Policy for disk balancing, `round_robin` or `least_used`."} })); // TODO: Add string column with custom volume-type-specific options setInMemoryMetadata(storage_metadata); diff --git a/src/Storages/System/StorageSystemSymbols.cpp b/src/Storages/System/StorageSystemSymbols.cpp index 56195544448..6b313407e61 100644 --- a/src/Storages/System/StorageSystemSymbols.cpp +++ b/src/Storages/System/StorageSystemSymbols.cpp @@ -22,9 +22,9 @@ StorageSystemSymbols::StorageSystemSymbols(const StorageID & table_id_) StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( { - {"symbol", std::make_shared()}, - {"address_begin", std::make_shared()}, - {"address_end", std::make_shared()}, + {"symbol", std::make_shared(), "Symbol name in the binary. It is mangled. You can apply demangle(symbol) to obtain a readable name."}, + {"address_begin", std::make_shared(), "Start address of the symbol in the binary."}, + {"address_end", std::make_shared(), "End address of the symbol in the binary."}, })); setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 6904eba8ab5..639c1455b83 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -67,9 +67,9 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) "Total number of uncompressed bytes, if it's possible to quickly determine the exact number " "of bytes from the part checksums for the table on storage, otherwise NULL (does not take underlying storage (if any) into account)." }, - {"parts", std::make_shared(std::make_shared())}, - {"active_parts", std::make_shared(std::make_shared())}, - {"total_marks", std::make_shared(std::make_shared())}, + {"parts", std::make_shared(std::make_shared()), "The total number of parts in this table."}, + {"active_parts", std::make_shared(std::make_shared()), "The number of active parts in this table."}, + {"total_marks", std::make_shared(std::make_shared()), "The total number of marks in all parts in this table."}, {"lifetime_rows", std::make_shared(std::make_shared()), "Total number of rows INSERTed since server start (only for Buffer tables)." }, diff --git a/src/Storages/System/StorageSystemTransactions.cpp b/src/Storages/System/StorageSystemTransactions.cpp index edc3739e713..295d93edf7d 100644 --- a/src/Storages/System/StorageSystemTransactions.cpp +++ b/src/Storages/System/StorageSystemTransactions.cpp @@ -23,14 +23,13 @@ static DataTypePtr getStateEnumType() ColumnsDescription StorageSystemTransactions::getColumnsDescription() { - /// TODO: Fill in all the comments. return ColumnsDescription { - {"tid", getTransactionIDDataType()}, - {"tid_hash", std::make_shared()}, - {"elapsed", std::make_shared()}, - {"is_readonly", std::make_shared()}, - {"state", getStateEnumType()}, + {"tid", getTransactionIDDataType(), "The identifier of the transaction."}, + {"tid_hash", std::make_shared(), "The hash of the identifier."}, + {"elapsed", std::make_shared(), "The amount of time the transaction being processed."}, + {"is_readonly", std::make_shared(), "The flag which shows whether the transaction has executed any write operation."}, + {"state", getStateEnumType(), "The state of the transaction. Possible values: RUNNING, COMMITTING, COMMITTED, ROLLED_BACK."}, }; } diff --git a/src/Storages/System/StorageSystemUserDirectories.cpp b/src/Storages/System/StorageSystemUserDirectories.cpp index 7b6c1144ae1..1b3469e7597 100644 --- a/src/Storages/System/StorageSystemUserDirectories.cpp +++ b/src/Storages/System/StorageSystemUserDirectories.cpp @@ -11,13 +11,12 @@ namespace DB { ColumnsDescription StorageSystemUserDirectories::getColumnsDescription() { - /// TODO: Fill in all the comments. return ColumnsDescription { - {"name", std::make_shared()}, - {"type", std::make_shared()}, - {"params", std::make_shared()}, - {"precedence", std::make_shared()}, + {"name", std::make_shared(), "The name of the directory."}, + {"type", std::make_shared(), "The type of the access storage e.g. users.xml or replicated or memory etc."}, + {"params", std::make_shared(), "JSON with the parameters of the access storage."}, + {"precedence", std::make_shared(), "The order in which this directory is declared in the config. The same order is used when ClickHouse tries to find a user or role."}, }; } diff --git a/src/Storages/System/StorageSystemUserProcesses.cpp b/src/Storages/System/StorageSystemUserProcesses.cpp index d36129aea63..65fbeedf406 100644 --- a/src/Storages/System/StorageSystemUserProcesses.cpp +++ b/src/Storages/System/StorageSystemUserProcesses.cpp @@ -18,10 +18,10 @@ ColumnsDescription StorageSystemUserProcesses::getColumnsDescription() { auto description = ColumnsDescription { - {"user", std::make_shared()}, - {"memory_usage", std::make_shared()}, - {"peak_memory_usage", std::make_shared()}, - {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"user", std::make_shared(), "User name."}, + {"memory_usage", std::make_shared(), "Sum of RAM used by all processes of the user. It might not include some types of dedicated memory. See the max_memory_usage setting."}, + {"peak_memory_usage", std::make_shared(), "The peak of memory usage of the user. It can be reset when no queries are run for the user."}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared()), "Summary of ProfileEvents that measure different metrics for the user. The description of them could be found in the table system.events"}, }; description.setAliases({ diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index 4734aeaaa82..0c34f04844d 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -69,10 +69,10 @@ ColumnsDescription StorageSystemUsers::getColumnsDescription() {"default_roles_except", std::make_shared(std::make_shared()), "All the granted roles set as default excepting of the listed ones." }, - {"grantees_any", std::make_shared()}, - {"grantees_list", std::make_shared(std::make_shared())}, - {"grantees_except", std::make_shared(std::make_shared())}, - {"default_database", std::make_shared()}, + {"grantees_any", std::make_shared(), "The flag that indicates whether a user with any grant option can grant it to anyone."}, + {"grantees_list", std::make_shared(std::make_shared()), "The list of users or roles to which this user is allowed to grant options to."}, + {"grantees_except", std::make_shared(std::make_shared()), "The list of users or roles to which this user is forbidden from grant options to."}, + {"default_database", std::make_shared(), "The name of the default database for this user."}, }; } diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index a0ed1d3c3d3..30539ed6b6a 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -36,15 +36,15 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() }, {"refresh_count", std::make_shared(), "Number of successful refreshes since last server restart or table creation."}, {"progress", std::make_shared(), "Progress of the current refresh, between 0 and 1."}, - {"elapsed", std::make_shared()}, - {"read_rows", std::make_shared(), "Number of rows read by the current refresh so far."}, - {"read_bytes", std::make_shared()}, + {"elapsed", std::make_shared(), "The amount of nanoseconds the current refresh took."}, + {"read_rows", std::make_shared(), "Number of rows read during the current refresh."}, + {"read_bytes", std::make_shared(), "Number of bytes read during the current refresh."}, {"total_rows", std::make_shared(), "Estimated total number of rows that need to be read by the current refresh."}, - {"total_bytes", std::make_shared()}, - {"written_rows", std::make_shared()}, - {"written_bytes", std::make_shared()}, - {"result_rows", std::make_shared()}, - {"result_bytes", std::make_shared()}, + {"total_bytes", std::make_shared(), "Estimated total number of bytes that need to be read by the current refresh."}, + {"written_rows", std::make_shared(), "Number of rows written during the current refresh."}, + {"written_bytes", std::make_shared(), "Number rof bytes written during the current refresh."}, + {"result_rows", std::make_shared(), "Estimated total number of rows in the result set of the SELECT query."}, + {"result_bytes", std::make_shared(), "Estimated total number of bytes in the result set of the SELECT query."}, }; } diff --git a/tests/queries/0_stateless/02294_system_certificates.reference b/tests/queries/0_stateless/02294_system_certificates.reference index 4655f996c40..432df9110a2 100644 --- a/tests/queries/0_stateless/02294_system_certificates.reference +++ b/tests/queries/0_stateless/02294_system_certificates.reference @@ -1,10 +1,10 @@ -version Int32 -serial_number Nullable(String) -signature_algo Nullable(String) -issuer Nullable(String) -not_before Nullable(String) -not_after Nullable(String) -subject Nullable(String) -pkey_algo Nullable(String) -path String -default UInt8 +version Int32 Version of the certificate. Values are 0 for v1, 1 for v2, 2 for v3. +serial_number Nullable(String) Serial Number of the certificate assigned by the issuer. +signature_algo Nullable(String) Signature Algorithm - an algorithm used by the issuer to sign this certificate. +issuer Nullable(String) Issuer - an unique identifier for the Certificate Authority issuing this certificate. +not_before Nullable(String) The beginning of the time window when this certificate is valid. +not_after Nullable(String) The end of the time window when this certificate is valid. +subject Nullable(String) Subject - identifies the owner of the public key. +pkey_algo Nullable(String) Public Key Algorithm defines the algorithm the public key can be used with. +path String Path to the file or directory containing this certificate. +default UInt8 Certificate is in the default certificate location. diff --git a/tests/queries/0_stateless/02326_settings_changes_system_table.reference b/tests/queries/0_stateless/02326_settings_changes_system_table.reference index 1c8c4fa1880..946b2727d30 100644 --- a/tests/queries/0_stateless/02326_settings_changes_system_table.reference +++ b/tests/queries/0_stateless/02326_settings_changes_system_table.reference @@ -1,3 +1,3 @@ -version String -changes Array(Tuple(\n name String,\n previous_value String,\n new_value String,\n reason String)) +version String The ClickHouse server version. +changes Array(Tuple(\n name String,\n previous_value String,\n new_value String,\n reason String)) The list of changes in settings which changed the behaviour of ClickHouse. 22.5 [('memory_overcommit_ratio_denominator','0','1073741824','Enable memory overcommit feature by default'),('memory_overcommit_ratio_denominator_for_user','0','1073741824','Enable memory overcommit feature by default')] diff --git a/tests/queries/0_stateless/02992_all_columns_should_have_comment.reference b/tests/queries/0_stateless/02992_all_columns_should_have_comment.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql new file mode 100644 index 00000000000..b984c12e285 --- /dev/null +++ b/tests/queries/0_stateless/02992_all_columns_should_have_comment.sql @@ -0,0 +1,4 @@ +SYSTEM FLUSH LOGS; +SELECT 'Column ' || name || ' from table ' || concat(database, '.', table) || ' should have a comment' +FROM system.columns +WHERE (database = 'system') AND (comment = '') AND (table NOT ILIKE '%_log_%') AND (table NOT IN ('numbers', 'numbers_mt', 'one')) AND (default_kind != 'ALIAS'); From c090c6fb0a6a91bb530654bf96bb31acb9d19f91 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 8 Mar 2024 18:52:15 +0000 Subject: [PATCH 41/66] Fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 987bf560add..bcedba7346d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2122,7 +2122,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis { /// Add row level security actions to prewhere. analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; - analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); + analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name; analysis.prewhere_info->row_level_filter->projectInput(false); analysis.filter_info = nullptr; } From b5289bf7abfe6367f88f0bfab77f39a4eae28438 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 8 Mar 2024 11:19:48 -0800 Subject: [PATCH 42/66] Reduce flakiness of 02932_refreshable_materialized_views (#60771) * Reduce flakiness of 02932_refreshable_materialized_views * Fix --- .../02932_refreshable_materialized_views.reference | 6 +++--- .../02932_refreshable_materialized_views.sh | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index b52d0847ff9..aa76806da9d 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -1,8 +1,8 @@ <1: created view> a [] 1 -CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 1 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x +CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x <2: refreshed> 3 1 1 -<3: time difference at least> 500 -<4: next refresh in> 1 +<3: time difference at least> 1000 +<4: next refresh in> 2 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` Int16\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 8daea063fc5..89942e25b67 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refre # Basic refreshing. $CLICKHOUSE_CLIENT -nq " create materialized view a - refresh after 1 second + refresh after 2 second engine Memory empty as select number as x from numbers(2) union all select rand64() as x" @@ -29,6 +29,7 @@ while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $ do sleep 0.1 done +start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" # Check table contents. $CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" # Wait for table contents to change. @@ -39,7 +40,6 @@ do [ "$res2" == "$res1" ] || break sleep 0.1 done -time2="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" # Wait for another change. while : do @@ -47,11 +47,11 @@ do [ "$res3" == "$res2" ] || break sleep 0.1 done -# Check that the two changes were at least 500ms apart, in particular that we're not refreshing +# Check that the two changes were at least 1 second apart, in particular that we're not refreshing # like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer # to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. $CLICKHOUSE_CLIENT -nq " - select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $time2, 500); + select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" # Create a source table from which views will read. @@ -61,7 +61,7 @@ $CLICKHOUSE_CLIENT -nq " # Switch to fake clock, change refresh schedule, change query. $CLICKHOUSE_CLIENT -nq " system test view a set fake time '2050-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03' ] do sleep 0.1 done From 6b4867f74656b0d090fda38432f2e4e8efe9106f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 8 Mar 2024 21:27:56 +0100 Subject: [PATCH 43/66] Debug fuzzer failures (#61062) --- docker/test/fuzzer/run-fuzzer.sh | 7 +++---- src/Parsers/ASTAlterQuery.cpp | 1 + src/Parsers/ASTCheckQuery.h | 1 + src/Parsers/ASTCreateIndexQuery.cpp | 1 + src/Parsers/ASTCreateQuery.cpp | 2 ++ src/Parsers/ASTDeleteQuery.cpp | 1 + src/Parsers/ASTDropIndexQuery.cpp | 1 + src/Parsers/ASTDropQuery.cpp | 1 + src/Parsers/ASTInsertQuery.cpp | 1 + src/Parsers/ASTOptimizeQuery.cpp | 1 + src/Parsers/ASTQueryWithTableAndOutput.h | 2 ++ src/Parsers/ASTRenameQuery.h | 2 ++ src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTUndropQuery.cpp | 1 + src/Parsers/ASTWatchQuery.h | 1 + 15 files changed, 20 insertions(+), 4 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 2834a632631..dc1b35b3a21 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -343,10 +343,9 @@ quit # which is confusing. task_exit_code=$fuzzer_exit_code echo "failure" > status.txt - { rg -ao "Found error:.*" fuzzer.log \ - || rg -ao "Exception:.*" fuzzer.log \ - || echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \ - | tail -1 > description.txt + echo "Achtung!" > description.txt + echo "Fuzzer went wrong with error code: ($fuzzer_exit_code). Its process died somehow when the server stayed alive. The server log probably won't tell you much so try to find information in other files." >>description.txt + { rg -ao "Found error:.*" fuzzer.log || rg -ao "Exception:.*" fuzzer.log; } | tail -1 >>description.txt fi if test -f core.*; then diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index a93ad1d1746..f104e715452 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -634,6 +634,7 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } else if (alter_object == AlterObjectType::DATABASE && database) diff --git a/src/Parsers/ASTCheckQuery.h b/src/Parsers/ASTCheckQuery.h index 5e9032e77fb..eca08b2b094 100644 --- a/src/Parsers/ASTCheckQuery.h +++ b/src/Parsers/ASTCheckQuery.h @@ -53,6 +53,7 @@ protected: settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp index 84493ff22f1..f6def3ed85c 100644 --- a/src/Parsers/ASTCreateIndexQuery.cpp +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -52,6 +52,7 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 1315ea5784c..de5eb40837f 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -337,6 +337,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); if (uuid != UUIDHelpers::Nil) @@ -370,6 +371,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); if (uuid != UUIDHelpers::Nil) diff --git a/src/Parsers/ASTDeleteQuery.cpp b/src/Parsers/ASTDeleteQuery.cpp index 1e8303dac62..67f3a85c9a5 100644 --- a/src/Parsers/ASTDeleteQuery.cpp +++ b/src/Parsers/ASTDeleteQuery.cpp @@ -40,6 +40,7 @@ void ASTDeleteQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); formatOnCluster(settings); diff --git a/src/Parsers/ASTDropIndexQuery.cpp b/src/Parsers/ASTDropIndexQuery.cpp index 1a1481ea27f..1109f32f019 100644 --- a/src/Parsers/ASTDropIndexQuery.cpp +++ b/src/Parsers/ASTDropIndexQuery.cpp @@ -47,6 +47,7 @@ void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatS settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index f41f28f745d..ca47ceccb85 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -76,6 +76,7 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index eb4bcce6578..72a569fe047 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -74,6 +74,7 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTOptimizeQuery.cpp b/src/Parsers/ASTOptimizeQuery.cpp index d9406a9eca0..397a37586fc 100644 --- a/src/Parsers/ASTOptimizeQuery.cpp +++ b/src/Parsers/ASTOptimizeQuery.cpp @@ -15,6 +15,7 @@ void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatSt settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); formatOnCluster(settings); diff --git a/src/Parsers/ASTQueryWithTableAndOutput.h b/src/Parsers/ASTQueryWithTableAndOutput.h index 1b8621fb63b..29dbd30e54b 100644 --- a/src/Parsers/ASTQueryWithTableAndOutput.h +++ b/src/Parsers/ASTQueryWithTableAndOutput.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -61,6 +62,7 @@ protected: settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } }; diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index 901f0121a28..73d12be094a 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -127,6 +127,7 @@ protected: settings.ostr << '.'; } + chassert(it->from.table); it->from.table->formatImpl(settings, state, frame); settings.ostr << (settings.hilite ? hilite_keyword : "") << (exchange ? " AND " : " TO ") << (settings.hilite ? hilite_none : ""); @@ -137,6 +138,7 @@ protected: settings.ostr << '.'; } + chassert(it->to.table); it->to.table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 63311a70e42..e2ebaee8438 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -114,6 +114,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); return settings.ostr; }; diff --git a/src/Parsers/ASTUndropQuery.cpp b/src/Parsers/ASTUndropQuery.cpp index ab96ca4711e..7212e264c0e 100644 --- a/src/Parsers/ASTUndropQuery.cpp +++ b/src/Parsers/ASTUndropQuery.cpp @@ -36,6 +36,7 @@ void ASTUndropQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTWatchQuery.h b/src/Parsers/ASTWatchQuery.h index 80b675f513f..a5b76c07605 100644 --- a/src/Parsers/ASTWatchQuery.h +++ b/src/Parsers/ASTWatchQuery.h @@ -52,6 +52,7 @@ protected: settings.ostr << '.'; } + chassert(table); table->formatImpl(settings, state, frame); if (is_watch_events) From 47b308d2346ef2b87d478d8ee29ee439097df000 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Mar 2024 08:42:33 +0100 Subject: [PATCH 44/66] Simplify bridges --- .../library-bridge/LibraryBridgeHandlers.cpp | 3 -- programs/odbc-bridge/ColumnInfoHandler.cpp | 2 - .../odbc-bridge/IdentifierQuoteHandler.cpp | 3 -- programs/odbc-bridge/ODBCBlockInputStream.cpp | 50 ++++++------------- .../odbc-bridge/ODBCBlockOutputStream.cpp | 1 - 5 files changed, 16 insertions(+), 43 deletions(-) diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index ab146f458df..26d887cfc98 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -1,6 +1,5 @@ #include "LibraryBridgeHandlers.h" -#include "CatBoostLibraryHandler.h" #include "CatBoostLibraryHandlerFactory.h" #include "Common/ProfileEvents.h" #include "ExternalDictionaryLibraryHandler.h" @@ -11,10 +10,8 @@ #include #include #include -#include #include #include -#include #include #include #include diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 774883657b7..4cb15de3b2c 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -8,13 +8,11 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include #include "getIdentifierQuote.h" diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index a23efb112de..cf5acdc4534 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -7,13 +7,10 @@ #include #include #include -#include -#include #include #include #include #include -#include #include "getIdentifierQuote.h" #include "validateODBCConnectionString.h" #include "ODBCPooledConnectionFactory.h" diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index c46144c3dc8..2cccc66a033 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -3,12 +3,8 @@ #include #include #include -#include -#include -#include #include #include -#include #include @@ -54,21 +50,7 @@ Chunk ODBCSource::generate() const auto & sample = description.sample_block.getByPosition(idx); if (!result.is_null(idx)) - { - bool is_nullable = description.types[idx].second; - - if (is_nullable) - { - ColumnNullable & column_nullable = assert_cast(*columns[idx]); - const auto & data_type = assert_cast(*sample.type); - insertValue(column_nullable.getNestedColumn(), data_type.getNestedType(), description.types[idx].first, result, idx); - column_nullable.getNullMapData().emplace_back(0); - } - else - { - insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); - } - } + insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); else insertDefaultValue(*columns[idx], *sample.column); } @@ -87,49 +69,49 @@ void ODBCSource::insertValue( switch (type) { case ValueType::vtUInt8: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt16: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt8: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt16: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFloat32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFloat64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFixedString:[[fallthrough]]; case ValueType::vtEnum8: case ValueType::vtEnum16: case ValueType::vtString: - assert_cast(column).insert(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUUID: { auto value = row.get(idx); - assert_cast(column).insert(parse(value.data(), value.size())); + column.insert(parse(value.data(), value.size())); break; } case ValueType::vtDate: - assert_cast(column).insertValue(UInt16{LocalDate{row.get(idx)}.getDayNum()}); + column.insert(UInt16{LocalDate{row.get(idx)}.getDayNum()}); break; case ValueType::vtDateTime: { @@ -139,7 +121,7 @@ void ODBCSource::insertValue( readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast(column).insertValue(static_cast(time)); + column.insert(static_cast(time)); break; } case ValueType::vtDateTime64: @@ -149,7 +131,7 @@ void ODBCSource::insertValue( DateTime64 time = 0; const auto * datetime_type = assert_cast(data_type.get()); readDateTime64Text(time, datetime_type->getScale(), in, datetime_type->getTimeZone()); - assert_cast(column).insertValue(time); + column.insert(time); break; } case ValueType::vtDecimal32: [[fallthrough]]; diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 87c09d1e757..37b70023169 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -1,6 +1,5 @@ #include "ODBCBlockOutputStream.h" -#include #include #include #include From 7f061ba89c5d598132c034cce980be714e9ef32c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sat, 9 Mar 2024 13:43:37 +0000 Subject: [PATCH 45/66] CI: add wf class in ci_config #do_not_test --- tests/ci/ci.py | 2 +- tests/ci/ci_config.py | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 6b754787d5e..1eec9a6771b 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1111,7 +1111,7 @@ def _configure_jobs( digests: Dict[str, str] = {} print("::group::Job Digests") - for job in CI_CONFIG.job_generator(): + for job in CI_CONFIG.job_generator(pr_info.head_ref): digest = job_digester.get_job_digest(CI_CONFIG.get_digest_config(job)) digests[job] = digest print(f" job [{job.rjust(50)}] has digest [{digest}]") diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 44dea116cbe..df8bfb1c2a8 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -11,6 +11,14 @@ from ci_utils import WithIter from integration_test_images import IMAGES +class WorkFlows(metaclass=WithIter): + PULL_REQUEST = "PULL_REQUEST" + MASTER = "MASTER" + BACKPORT = "BACKPORT" + RELEASE = "RELEASE" + SYNC = "SYNC" + + class CIStages(metaclass=WithIter): NA = "UNKNOWN" BUILDS_1 = "Builds_1" @@ -694,10 +702,11 @@ class CIConfig: ), f"Invalid check_name or CI_CONFIG outdated, config not found for [{check_name}]" return res # type: ignore - def job_generator(self) -> Iterable[str]: + def job_generator(self, branch: str) -> Iterable[str]: """ traverses all check names in CI pipeline """ + assert branch for config in ( self.other_jobs_configs, self.build_config, From 4af16043b86a6e72b3f13579a938d271fd0b77dd Mon Sep 17 00:00:00 2001 From: tomershafir Date: Sat, 9 Mar 2024 21:27:28 +0200 Subject: [PATCH 46/66] improve randomness by erase buf elem when full --- src/Client/QueryFuzzer.cpp | 49 +++++++++++++++++++------------------- src/Client/QueryFuzzer.h | 6 ++--- 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index ea2e5e17afd..d23b5159854 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -232,7 +232,7 @@ ASTPtr QueryFuzzer::getRandomColumnLike() return nullptr; } - ASTPtr new_ast = column_like[fuzz_rand() % column_like.size()]->clone(); + ASTPtr new_ast = column_like[fuzz_rand() % column_like.size()].second->clone(); new_ast->setAlias(""); return new_ast; @@ -272,7 +272,7 @@ void QueryFuzzer::replaceWithTableLike(ASTPtr & ast) return; } - ASTPtr new_ast = table_like[fuzz_rand() % table_like.size()]->clone(); + ASTPtr new_ast = table_like[fuzz_rand() % table_like.size()].second->clone(); std::string old_alias = ast->tryGetAlias(); new_ast->setAlias(old_alias); @@ -1214,51 +1214,46 @@ void QueryFuzzer::fuzz(ASTPtr & ast) } } +#define AST_FUZZER_PART_TYPE_CAP 1000 + /* * This functions collects various parts of query that we can then substitute * to a query being fuzzed. - * - * TODO: we just stop remembering new parts after our corpus reaches certain size. - * This is boring, should implement a random replacement of existing parst with - * small probability. Do this after we add this fuzzer to CI and fix all the - * problems it can routinely find even in this boring version. */ void QueryFuzzer::collectFuzzInfoMain(ASTPtr ast) { collectFuzzInfoRecurse(ast); - - column_like.clear(); - for (const auto & [name, value] : column_like_map) - { - column_like.push_back(value); - } - - table_like.clear(); - for (const auto & [name, value] : table_like_map) - { - table_like.push_back(value); - } } void QueryFuzzer::addTableLike(ASTPtr ast) { - if (table_like_map.size() > 1000) + if (table_like_map.size() > AST_FUZZER_PART_TYPE_CAP) { - table_like_map.clear(); + const auto iter = std::next(table_like.begin(), fuzz_rand() % table_like.size()); + const auto ast_del = *iter; + table_like.erase(iter); + table_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); if (name.size() < 200) { - table_like_map.insert({name, ast}); + const auto res = table_like_map.insert({name, ast}); + if (res.second) + { + table_like.push_back({name, ast}); + } } } void QueryFuzzer::addColumnLike(ASTPtr ast) { - if (column_like_map.size() > 1000) + if (column_like_map.size() > AST_FUZZER_PART_TYPE_CAP) { - column_like_map.clear(); + const auto iter = std::next(column_like.begin(), fuzz_rand() % column_like.size()); + const auto ast_del = *iter; + column_like.erase(iter); + column_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); @@ -1273,7 +1268,11 @@ void QueryFuzzer::addColumnLike(ASTPtr ast) } if (name.size() < 200) { - column_like_map.insert({name, ast}); + const auto res = column_like_map.insert({name, ast}); + if (res.second) + { + column_like.push_back({name, ast}); + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index 3bc7b0842d3..6165e589cae 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -50,12 +50,12 @@ struct QueryFuzzer // we are currently fuzzing. We add some part from each new query we are asked // to fuzz, and keep this state between queries, so the fuzzing output becomes // more interesting over time, as the queries mix. - // The maps are used for collection, and the vectors are used for random access. + // The hash tables are used for collection, and the vectors are used for random access. std::unordered_map column_like_map; - std::vector column_like; + std::vector> column_like; std::unordered_map table_like_map; - std::vector table_like; + std::vector> table_like; // Some debug fields for detecting problematic ASTs with loops. // These are reset for each fuzzMain call. From 681bd28c877a365b8f18f6325c00e697ee83baef Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sat, 9 Mar 2024 20:38:23 +0000 Subject: [PATCH 47/66] CI: make style check fast #do_not_test --- docker/test/style/Dockerfile | 1 - tests/ci/style_check.py | 41 ++++++++++++++----- utils/check-style/check-pylint | 15 +++++++ utils/check-style/check-style | 10 ----- utils/check-style/check_cpp_docs.sh | 34 +++++++++++++++ utils/check-style/check_py.sh | 17 ++++++++ .../process_style_check_result.py | 7 ++-- 7 files changed, 100 insertions(+), 25 deletions(-) create mode 100755 utils/check-style/check-pylint create mode 100755 utils/check-style/check_cpp_docs.sh create mode 100755 utils/check-style/check_py.sh rename {docker/test/style => utils/check-style}/process_style_check_result.py (96%) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 122f558bab2..b4ffcfb597c 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -60,5 +60,4 @@ RUN arch=${TARGETARCH:-amd64} \ COPY run.sh / -COPY process_style_check_result.py / CMD ["/bin/bash", "/run.sh"] diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index d004f3ed215..d0565e136d3 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 import argparse +from concurrent.futures import ProcessPoolExecutor import csv import logging import os @@ -119,7 +120,7 @@ def checkout_last_ref(pr_info: PRInfo) -> None: def main(): logging.basicConfig(level=logging.INFO) logging.getLogger("git_helper").setLevel(logging.DEBUG) - args = parse_args() + # args = parse_args() stopwatch = Stopwatch() @@ -127,28 +128,46 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - pr_info = PRInfo() + # pr_info = PRInfo() IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd = ( + cmd_1 = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " - f"{image}" + f"--entrypoint= -w/ClickHouse/utils/check-style " + f"{image} ./check_cpp_docs.sh" ) + cmd_2 = ( + f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " + f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " + f"--entrypoint= -w/ClickHouse/utils/check-style " + f"{image} ./check_py.sh" + ) + logging.info("Is going to run the command: %s", cmd_1) + logging.info("Is going to run the command: %s", cmd_2) - if args.push: - checkout_head(pr_info) + with ProcessPoolExecutor(max_workers=2) as executor: + # Submit commands for execution in parallel + future1 = executor.submit(subprocess.run, cmd_1, shell=True) + future2 = executor.submit(subprocess.run, cmd_2, shell=True) + # Wait for both commands to complete + _ = future1.result() + _ = future2.result() + + # if args.push: + # checkout_head(pr_info) - logging.info("Is going to run the command: %s", cmd) subprocess.check_call( - cmd, + f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " + f"--out-results-file {temp_path}/test_results.tsv --out-status-file {temp_path}/check_status.tsv || " + f'echo -e "failure\tCannot parse results" > {temp_path}/check_status.tsv', shell=True, ) - if args.push: - commit_push_staged(pr_info) - checkout_last_ref(pr_info) + # if args.push: + # commit_push_staged(pr_info) + # checkout_last_ref(pr_info) state, description, test_results, additional_files = process_result(temp_path) diff --git a/utils/check-style/check-pylint b/utils/check-style/check-pylint new file mode 100755 index 00000000000..7959a414023 --- /dev/null +++ b/utils/check-style/check-pylint @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + + +LC_ALL="en_US.UTF-8" +ROOT_PATH=$(git rev-parse --show-toplevel) + +function xargs-pylint { + # $1 is number maximum arguments per pylint process + sort | awk '$2=="text/x-script.python" {print $1}' | \ + xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n +} + +find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 +# Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo +find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 3a5d0c053ea..d7387c3f843 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -152,16 +152,6 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | grep -vP $EXCLUDE_DIRS | xargs xmllint --noout --nonet -function xargs-pylint { - # $1 is number maximum arguments per pylint process - sort | awk '$2=="text/x-script.python" {print $1}' | \ - xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n -} - -find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 -# Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo -find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 - find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | xargs yamllint --config-file=$ROOT_PATH/.yamllint diff --git a/utils/check-style/check_cpp_docs.sh b/utils/check-style/check_cpp_docs.sh new file mode 100755 index 00000000000..7ad3cede758 --- /dev/null +++ b/utils/check-style/check_cpp_docs.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +# yaml check is not the best one + +cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv + +# FIXME: 30 sec to wait +# echo "Check duplicates" | ts +# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt + +echo "Check style" | ts +./check-style -n |& tee /test_output/style_output.txt +echo "Check typos" | ts +./check-typos |& tee /test_output/typos_output.txt +echo "Check docs spelling" | ts +./check-doc-aspell |& tee /test_output/docs_spelling_output.txt +echo "Check whitespaces" | ts +./check-whitespaces -n |& tee /test_output/whitespaces_output.txt +echo "Check workflows" | ts +./check-workflows |& tee /test_output/workflows_output.txt +echo "Check submodules" | ts +./check-submodules |& tee /test_output/submodules_output.txt +echo "Check style. Done" | ts + +# FIXME: 6 min to wait +# echo "Check shell scripts with shellcheck" | ts +# ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt + + +# FIXME: move out +# /process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv +# echo "Check help for changelog generator works" | ts +# cd ../changelog || exit 1 +# ./changelog.py -h 2>/dev/null 1>&2 diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh new file mode 100755 index 00000000000..48c02013734 --- /dev/null +++ b/utils/check-style/check_py.sh @@ -0,0 +1,17 @@ +#!/bin/bash + +# yaml check is not the best one + +cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv + +# FIXME: 1 min to wait + head checkout +# echo "Check python formatting with black" | ts +# ./check-black -n |& tee /test_output/black_output.txt + +echo "Check pylint" | ts +./check-pylint -n |& tee /test_output/pylint_output.txt +echo "Check pylint. Done" | ts + +echo "Check python type hinting with mypy" | ts +./check-mypy -n |& tee /test_output/mypy_output.txt +echo "Check python type hinting with mypy. Done" | ts diff --git a/docker/test/style/process_style_check_result.py b/utils/check-style/process_style_check_result.py similarity index 96% rename from docker/test/style/process_style_check_result.py rename to utils/check-style/process_style_check_result.py index bc06df1af31..7980c01dd37 100755 --- a/docker/test/style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -13,10 +13,11 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - "duplicate includes", - "shellcheck", + #"duplicate includes", + #"shellcheck", "style", - "black", + "pylint", + #"black", "mypy", "typos", "whitespaces", From 453eb259842f3181d5f9329e0e82e4c1e66f18b0 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Sun, 10 Mar 2024 08:42:54 +0200 Subject: [PATCH 48/66] rm trailing whitespace --- src/Client/QueryFuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index d23b5159854..7f1dce4b29a 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1232,7 +1232,7 @@ void QueryFuzzer::addTableLike(ASTPtr ast) const auto iter = std::next(table_like.begin(), fuzz_rand() % table_like.size()); const auto ast_del = *iter; table_like.erase(iter); - table_like_map.erase(ast_del.first); + table_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); From 9ddec346a03108198fea011c1113495050ac628b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:02:34 +0100 Subject: [PATCH 49/66] Miscellaneous --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 8 +- src/Storages/MergeTree/MergeTask.cpp | 4 +- .../test_odbc_interaction/test_exiled.py | 111 ------------------ 3 files changed, 6 insertions(+), 117 deletions(-) delete mode 100644 tests/integration/test_odbc_interaction/test_exiled.py diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 2cccc66a033..4fd9b4cca45 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -98,7 +98,7 @@ void ODBCSource::insertValue( case ValueType::vtFloat64: column.insert(row.get(idx)); break; - case ValueType::vtFixedString:[[fallthrough]]; + case ValueType::vtFixedString: case ValueType::vtEnum8: case ValueType::vtEnum16: case ValueType::vtString: @@ -134,9 +134,9 @@ void ODBCSource::insertValue( column.insert(time); break; } - case ValueType::vtDecimal32: [[fallthrough]]; - case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: [[fallthrough]]; + case ValueType::vtDecimal32: + case ValueType::vtDecimal64: + case ValueType::vtDecimal128: case ValueType::vtDecimal256: { auto value = row.get(idx); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index aa38198334e..06b6927bc43 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -296,7 +296,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() switch (global_ctx->chosen_merge_algorithm) { - case MergeAlgorithm::Horizontal : + case MergeAlgorithm::Horizontal: { global_ctx->merging_columns = global_ctx->storage_columns; global_ctx->merging_column_names = global_ctx->all_column_names; @@ -304,7 +304,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->gathering_column_names.clear(); break; } - case MergeAlgorithm::Vertical : + case MergeAlgorithm::Vertical: { ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream(); ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); diff --git a/tests/integration/test_odbc_interaction/test_exiled.py b/tests/integration/test_odbc_interaction/test_exiled.py deleted file mode 100644 index bdc819b8221..00000000000 --- a/tests/integration/test_odbc_interaction/test_exiled.py +++ /dev/null @@ -1,111 +0,0 @@ -import time -import logging -import pytest - -from helpers.cluster import ClickHouseCluster, assert_eq_with_retry -from test_odbc_interaction.test import ( - create_mysql_db, - create_mysql_table, - get_mysql_conn, - skip_test_msan, -) - - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - with_odbc_drivers=True, - main_configs=["configs/openssl.xml", "configs/odbc_logging.xml"], - stay_alive=True, - dictionaries=["configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml"], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] - logging.debug(f"sqlite data received: {sqlite_db}") - node1.exec_in_container( - [ - "sqlite3", - sqlite_db, - "CREATE TABLE t2(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);", - ], - privileged=True, - user="root", - ) - - node1.exec_in_container( - ["sqlite3", sqlite_db, "INSERT INTO t2 values(1, 1, 2, 3);"], - privileged=True, - user="root", - ) - - node1.query("SYSTEM RELOAD DICTIONARY sqlite3_odbc_hashed") - - yield cluster - except Exception as ex: - logging.exception(ex) - raise ex - finally: - cluster.shutdown() - - -# This test kills ClickHouse server and ODBC bridge and in worst scenario -# may cause group test crashes. Thus, this test is executed in a separate "module" -# with separate environment. -def test_bridge_dies_with_parent(started_cluster): - skip_test_msan(node1) - - if node1.is_built_with_address_sanitizer(): - # TODO: Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge in this test and - # that's linked somehow with that we have replaced getauxval() in glibc-compatibility. - # The leak sanitizer calls getauxval() for its own purposes, and our replaced version doesn't seem to be equivalent in that case. - pytest.skip( - "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" - ) - - assert_eq_with_retry( - node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(1))", "3" - ) - - clickhouse_pid = node1.get_process_pid("clickhouse server") - bridge_pid = node1.get_process_pid("odbc-bridge") - assert clickhouse_pid is not None - assert bridge_pid is not None - - try: - node1.exec_in_container( - ["kill", str(clickhouse_pid)], privileged=True, user="root" - ) - except: - pass - - for _ in range(30): - time.sleep(1) - clickhouse_pid = node1.get_process_pid("clickhouse server") - if clickhouse_pid is None: - break - - for _ in range(30): - time.sleep(1) # just for sure, that odbc-bridge caught signal - bridge_pid = node1.get_process_pid("odbc-bridge") - if bridge_pid is None: - break - - if bridge_pid: - out = node1.exec_in_container( - ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], - privileged=True, - user="root", - ) - logging.debug(f"Bridge is running, gdb output:\n{out}") - - try: - assert clickhouse_pid is None - assert bridge_pid is None - finally: - node1.start_clickhouse(20) From 1a2d403f7b7dc16676d1b550dd346c06503c550c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:16:21 +0100 Subject: [PATCH 50/66] Miscellaneae --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 4fd9b4cca45..934639ae604 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -118,7 +118,8 @@ void ODBCSource::insertValue( auto value = row.get(idx); ReadBufferFromString in(value); time_t time = 0; - readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); + const DataTypeDateTime & datetime_type = assert_cast(*data_type); + readDateTimeText(time, in, datetime_type.getTimeZone()); if (time < 0) time = 0; column.insert(static_cast(time)); @@ -129,8 +130,8 @@ void ODBCSource::insertValue( auto value = row.get(idx); ReadBufferFromString in(value); DateTime64 time = 0; - const auto * datetime_type = assert_cast(data_type.get()); - readDateTime64Text(time, datetime_type->getScale(), in, datetime_type->getTimeZone()); + const DataTypeDateTime64 & datetime_type = assert_cast(*data_type); + readDateTime64Text(time, datetime_type.getScale(), in, datetime_type.getTimeZone()); column.insert(time); break; } From 157adefc70d8152f744b23152309cbf03b00c990 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:17:29 +0100 Subject: [PATCH 51/66] Fix error --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 934639ae604..59a5deac960 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -50,7 +50,7 @@ Chunk ODBCSource::generate() const auto & sample = description.sample_block.getByPosition(idx); if (!result.is_null(idx)) - insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); + insertValue(*columns[idx], removeNullable(sample.type), description.types[idx].first, result, idx); else insertDefaultValue(*columns[idx], *sample.column); } From 3b8a8c9649b00d360c5d6169dc769c5371dd44a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:28:30 +0100 Subject: [PATCH 52/66] Better gitignore --- docker/test/stateless/.gitignore | 1 + programs/server/.gitignore | 1 + 2 files changed, 2 insertions(+) create mode 100644 docker/test/stateless/.gitignore diff --git a/docker/test/stateless/.gitignore b/docker/test/stateless/.gitignore new file mode 100644 index 00000000000..928fed26d6d --- /dev/null +++ b/docker/test/stateless/.gitignore @@ -0,0 +1 @@ +/minio_data diff --git a/programs/server/.gitignore b/programs/server/.gitignore index ddc480e4b29..34a774bde9d 100644 --- a/programs/server/.gitignore +++ b/programs/server/.gitignore @@ -2,6 +2,7 @@ /metadata_dropped /data /store +/disks /access /flags /dictionaries_lib From d8e3fa6dc107a4b43edb091a3688b1672a6c2f70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:35:57 +0100 Subject: [PATCH 53/66] Fix something strange --- src/Interpreters/fuzzers/execute_query_fuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index 6f84a60f2af..edff202d547 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -3,7 +3,7 @@ #include #include "Processors/Executors/PullingPipelineExecutor.h" -#include +#include #include #include #include From 5c41727725eb1066ff2d86fc5da3e272e03155c1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 3 Mar 2024 14:22:40 +0100 Subject: [PATCH 54/66] http connections pools --- .../Net/include/Poco/Net/HTTPChunkedStream.h | 4 + .../Net/include/Poco/Net/HTTPClientSession.h | 24 +- .../include/Poco/Net/HTTPFixedLengthStream.h | 4 + base/poco/Net/include/Poco/Net/HTTPSession.h | 24 +- base/poco/Net/include/Poco/Net/HTTPStream.h | 2 + base/poco/Net/src/HTTPChunkedStream.cpp | 4 +- base/poco/Net/src/HTTPClientSession.cpp | 23 +- base/poco/Net/src/HTTPFixedLengthStream.cpp | 6 + programs/server/Server.cpp | 18 + .../static-files-disk-uploader.cpp | 6 +- src/Access/Common/AccessType.h | 3 +- src/Access/HTTPAuthClient.h | 2 +- .../CatBoostLibraryBridgeHelper.cpp | 104 +- .../ExternalDictionaryLibraryBridgeHelper.cpp | 38 +- src/BridgeHelper/XDBCBridgeHelper.h | 26 +- src/Client/Connection.cpp | 16 + src/Common/CurrentMetrics.cpp | 12 + src/Common/DNSResolver.cpp | 3 +- src/Common/HTTPConnectionPool.cpp | 865 ++++++++++++++++ src/Common/HTTPConnectionPool.h | 91 ++ src/Common/HostResolvePool.cpp | 293 ++++++ src/Common/HostResolvePool.h | 218 ++++ src/Common/ProfileEvents.cpp | 29 + src/Common/ProxyConfiguration.h | 12 +- src/Common/ProxyListConfigurationResolver.cpp | 2 - .../RemoteProxyConfigurationResolver.cpp | 2 +- src/Common/tests/gtest_connection_pool.cpp | 558 +++++++++++ src/Common/tests/gtest_resolve_pool.cpp | 278 ++++++ src/Core/ServerSettings.h | 11 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 3 +- src/Dictionaries/HTTPDictionarySource.cpp | 91 +- src/Dictionaries/XDBCDictionarySource.cpp | 15 +- src/Dictionaries/XDBCDictionarySource.h | 2 +- src/Disks/IO/ReadBufferFromWebServer.cpp | 81 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 - .../ObjectStorages/Web/WebObjectStorage.cpp | 35 +- src/Functions/sqid.cpp | 2 +- src/IO/ConnectionTimeouts.cpp | 15 + src/IO/ConnectionTimeouts.h | 4 + src/IO/HTTPCommon.cpp | 379 +------ src/IO/HTTPCommon.h | 48 +- src/IO/LimitReadBuffer.cpp | 25 +- src/IO/LimitReadBuffer.h | 18 +- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 +- src/IO/MMapReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadBuffer.h | 12 +- src/IO/ReadBufferFromFileDescriptor.cpp | 4 +- src/IO/ReadBufferFromFileDescriptor.h | 4 +- src/IO/ReadBufferFromIStream.cpp | 40 +- src/IO/ReadBufferFromS3.cpp | 100 +- src/IO/ReadBufferFromS3.h | 6 +- src/IO/ReadSettings.h | 2 +- src/IO/ReadWriteBufferFromHTTP.cpp | 945 +++++++----------- src/IO/ReadWriteBufferFromHTTP.h | 426 ++++---- src/IO/S3/PocoHTTPClient.cpp | 63 +- src/IO/S3/PocoHTTPClient.h | 17 +- src/IO/SeekableReadBuffer.h | 2 +- src/IO/{S3 => }/SessionAwareIOStream.h | 2 +- src/IO/WriteBufferFromHTTP.cpp | 6 +- src/IO/WriteBufferFromHTTP.h | 3 +- src/IO/copyData.cpp | 2 +- src/Interpreters/Context.cpp | 13 - src/Interpreters/Context.h | 4 - src/Interpreters/DDLTask.cpp | 5 + src/Interpreters/InterpreterSystemQuery.cpp | 9 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + .../Formats/Impl/AvroRowInputFormat.cpp | 4 +- src/Storages/MergeTree/DataPartsExchange.cpp | 25 +- src/Storages/MergeTree/DataPartsExchange.h | 10 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 49 +- src/TableFunctions/ITableFunctionXDBC.cpp | 17 +- .../test_cancel_backup.py | 2 +- .../configs/setting.xml | 1 + .../configs/storage_conf.xml | 1 + .../test_disk_over_web_server/test.py | 2 +- tests/integration/test_dns_cache/test.py | 12 +- tests/integration/test_http_failover/test.py | 6 +- .../test_redirect_url_storage/test.py | 2 +- .../test_s3_table_functions/test.py | 1 + .../s3_mocks/unstable_server.py | 2 +- .../0_stateless/00646_url_engine.python | 13 +- .../01271_show_privileges.reference | 1 + .../0_stateless/01293_show_settings.reference | 1 + .../0_stateless/02205_HTTP_user_agent.python | 13 +- .../0_stateless/02233_HTTP_ranged.python | 25 +- ...rofile_events_from_query_log_and_client.sh | 2 +- ...89_reading_from_s3_with_connection_pool.sh | 6 +- .../02833_url_without_path_encoding.sh | 4 +- 92 files changed, 3507 insertions(+), 1771 deletions(-) create mode 100644 src/Common/HTTPConnectionPool.cpp create mode 100644 src/Common/HTTPConnectionPool.h create mode 100644 src/Common/HostResolvePool.cpp create mode 100644 src/Common/HostResolvePool.h create mode 100644 src/Common/tests/gtest_connection_pool.cpp create mode 100644 src/Common/tests/gtest_resolve_pool.cpp rename src/IO/{S3 => }/SessionAwareIOStream.h (97%) diff --git a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h index 5f4729c9278..a6576aa561d 100644 --- a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h @@ -45,6 +45,8 @@ namespace Net ~HTTPChunkedStreamBuf(); void close(); + bool isComplete() const { return _chunk == std::char_traits::eof(); } + protected: int readFromDevice(char * buffer, std::streamsize length); int writeToDevice(const char * buffer, std::streamsize length); @@ -68,6 +70,8 @@ namespace Net ~HTTPChunkedIOS(); HTTPChunkedStreamBuf * rdbuf(); + bool isComplete() const { return _buf.isComplete(); } + protected: HTTPChunkedStreamBuf _buf; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index 7c0caa1c18b..1cef988566c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -210,7 +210,7 @@ namespace Net void setKeepAliveTimeout(const Poco::Timespan & timeout); /// Sets the connection timeout for HTTP connections. - const Poco::Timespan & getKeepAliveTimeout() const; + Poco::Timespan getKeepAliveTimeout() const; /// Returns the connection timeout for HTTP connections. virtual std::ostream & sendRequest(HTTPRequest & request); @@ -275,7 +275,7 @@ namespace Net /// This method should only be called if the request contains /// a "Expect: 100-continue" header. - void flushRequest(); + virtual void flushRequest(); /// Flushes the request stream. /// /// Normally this method does not need to be called. @@ -283,7 +283,7 @@ namespace Net /// fully sent if receiveResponse() is not called, e.g., /// because the underlying socket will be detached. - void reset(); + virtual void reset(); /// Resets the session and closes the socket. /// /// The next request will initiate a new connection, @@ -303,6 +303,9 @@ namespace Net /// Returns true if the proxy should be bypassed /// for the current host. + const Poco::Timestamp & getLastRequest() const; + /// Returns time when connection has been used last time + protected: enum { @@ -338,6 +341,10 @@ namespace Net /// Calls proxyConnect() and attaches the resulting StreamSocket /// to the HTTPClientSession. + void setLastRequest(Poco::Timestamp time); + + void assign(HTTPClientSession & session); + HTTPSessionFactory _proxySessionFactory; /// Factory to create HTTPClientSession to proxy. private: @@ -433,11 +440,20 @@ namespace Net } - inline const Poco::Timespan & HTTPClientSession::getKeepAliveTimeout() const + inline Poco::Timespan HTTPClientSession::getKeepAliveTimeout() const { return _keepAliveTimeout; } + inline const Poco::Timestamp & HTTPClientSession::getLastRequest() const + { + return _lastRequest; + } + + inline void HTTPClientSession::setLastRequest(Poco::Timestamp time) + { + _lastRequest = time; + } } } // namespace Poco::Net diff --git a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h index 2f4df102605..17fa47cfa9b 100644 --- a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h @@ -48,6 +48,8 @@ namespace Net HTTPFixedLengthStreamBuf(HTTPSession & session, ContentLength length, openmode mode); ~HTTPFixedLengthStreamBuf(); + bool isComplete() const; + protected: int readFromDevice(char * buffer, std::streamsize length); int writeToDevice(const char * buffer, std::streamsize length); @@ -67,6 +69,8 @@ namespace Net ~HTTPFixedLengthIOS(); HTTPFixedLengthStreamBuf * rdbuf(); + bool isComplete() const { return _buf.isComplete(); } + protected: HTTPFixedLengthStreamBuf _buf; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index 934b34be5d5..cac14f479db 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -64,6 +64,15 @@ namespace Net Poco::Timespan getTimeout() const; /// Returns the timeout for the HTTP session. + Poco::Timespan getConnectionTimeout() const; + /// Returns connection timeout for the HTTP session. + + Poco::Timespan getSendTimeout() const; + /// Returns send timeout for the HTTP session. + + Poco::Timespan getReceiveTimeout() const; + /// Returns receive timeout for the HTTP session. + bool connected() const; /// Returns true if the underlying socket is connected. @@ -217,12 +226,25 @@ namespace Net return _keepAlive; } - inline Poco::Timespan HTTPSession::getTimeout() const { return _receiveTimeout; } + inline Poco::Timespan HTTPSession::getConnectionTimeout() const + { + return _connectionTimeout; + } + + inline Poco::Timespan HTTPSession::getSendTimeout() const + { + return _sendTimeout; + } + + inline Poco::Timespan HTTPSession::getReceiveTimeout() const + { + return _receiveTimeout; + } inline StreamSocket & HTTPSession::socket() { diff --git a/base/poco/Net/include/Poco/Net/HTTPStream.h b/base/poco/Net/include/Poco/Net/HTTPStream.h index 48502347b2c..a00a861880f 100644 --- a/base/poco/Net/include/Poco/Net/HTTPStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPStream.h @@ -63,6 +63,8 @@ namespace Net ~HTTPIOS(); HTTPStreamBuf * rdbuf(); + bool isComplete() const { return false; } + protected: HTTPStreamBuf _buf; }; diff --git a/base/poco/Net/src/HTTPChunkedStream.cpp b/base/poco/Net/src/HTTPChunkedStream.cpp index 376e3f55492..16ed1e71c31 100644 --- a/base/poco/Net/src/HTTPChunkedStream.cpp +++ b/base/poco/Net/src/HTTPChunkedStream.cpp @@ -49,10 +49,12 @@ HTTPChunkedStreamBuf::~HTTPChunkedStreamBuf() void HTTPChunkedStreamBuf::close() { - if (_mode & std::ios::out) + if (_mode & std::ios::out && _chunk != std::char_traits::eof()) { sync(); _session.write("0\r\n\r\n", 5); + + _chunk = std::char_traits::eof(); } } diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index 2282cca682b..33a3dcc4901 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -227,7 +227,7 @@ void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout) std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) { _pRequestStream = 0; - _pResponseStream = 0; + _pResponseStream = 0; clearException(); _responseReceived = false; @@ -501,5 +501,26 @@ bool HTTPClientSession::bypassProxy() const else return false; } +void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session) +{ + poco_assert (this != &session); + + if (session.buffered()) + throw Poco::LogicException("assign a session with not empty buffered data"); + + if (buffered()) + throw Poco::LogicException("assign to a session with not empty buffered data"); + + attachSocket(session.detachSocket()); + setLastRequest(session.getLastRequest()); + setResolvedHost(session.getResolvedHost()); + setKeepAlive(session.getKeepAlive()); + + setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout()); + setKeepAliveTimeout(session.getKeepAliveTimeout()); + setProxyConfig(session.getProxyConfig()); + + session.reset(); +} } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPFixedLengthStream.cpp b/base/poco/Net/src/HTTPFixedLengthStream.cpp index fd77ff71cd9..837e5723c57 100644 --- a/base/poco/Net/src/HTTPFixedLengthStream.cpp +++ b/base/poco/Net/src/HTTPFixedLengthStream.cpp @@ -43,6 +43,12 @@ HTTPFixedLengthStreamBuf::~HTTPFixedLengthStreamBuf() } +bool HTTPFixedLengthStreamBuf::isComplete() const +{ + return _count == _length; +} + + int HTTPFixedLengthStreamBuf::readFromDevice(char* buffer, std::streamsize length) { int n = 0; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f2c5dcc0f6d..7636f221ab5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include #include @@ -1547,6 +1548,23 @@ try FileCacheFactory::instance().updateSettingsFromConfig(*config); + HTTPConnectionPools::instance().setLimits( + HTTPConnectionPools::Limits{ + new_server_settings.disk_connections_soft_limit, + new_server_settings.disk_connections_warn_limit, + new_server_settings.disk_connections_store_limit, + }, + HTTPConnectionPools::Limits{ + new_server_settings.storage_connections_soft_limit, + new_server_settings.storage_connections_warn_limit, + new_server_settings.storage_connections_store_limit, + }, + HTTPConnectionPools::Limits{ + new_server_settings.http_connections_soft_limit, + new_server_settings.http_connections_warn_limit, + new_server_settings.http_connections_store_limit, + }); + ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. diff --git a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp index d54a2d2da6e..dfe68c819fc 100644 --- a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp +++ b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp @@ -65,7 +65,7 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes /// test mode for integration tests. if (test_mode) - dst_buf = std::make_shared(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); + dst_buf = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); else dst_buf = std::make_shared(dst_file_path); @@ -88,7 +88,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ { dst_path /= "store"; auto files_root = dst_path / prefix; - root_meta = std::make_shared(Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); + root_meta = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); } else { @@ -112,7 +112,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ if (test_mode) { auto files_root = dst_path / prefix; - directory_meta = std::make_shared(Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); + directory_meta = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); } else { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 87f96ca48be..6095f8ce6d3 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -154,7 +154,8 @@ enum class AccessType M(SET_DEFINER, "", USER_NAME, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ - M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MMAP_CACHE, "SYSTEM DROP MMAP, DROP MMAP CACHE, DROP MMAP", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/src/Access/HTTPAuthClient.h b/src/Access/HTTPAuthClient.h index caefe869005..a8b56cf05a7 100644 --- a/src/Access/HTTPAuthClient.h +++ b/src/Access/HTTPAuthClient.h @@ -36,7 +36,7 @@ public: Result authenticateRequest(Poco::Net::HTTPRequest & request) const { - auto session = makeHTTPSession(uri, timeouts); + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts); Poco::Net::HTTPResponse response; auto milliseconds_to_wait = retry_initial_backoff_ms; diff --git a/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp b/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp index 4db1eb720ac..bb9d890a691 100644 --- a/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp +++ b/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp @@ -58,8 +58,12 @@ bool CatBoostLibraryBridgeHelper::bridgeHandShake() String result; try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials); - readString(result, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(http_timeouts) + .create(credentials); + + readString(result, *buf); } catch (...) { @@ -79,29 +83,29 @@ ExternalModelInfos CatBoostLibraryBridgeHelper::listModels() { startBridgeSync(); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_LIST_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [](std::ostream &) {}, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIST_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .create(credentials); ExternalModelInfos result; UInt64 num_rows; - readIntBinary(num_rows, buf); + readIntBinary(num_rows, *buf); for (UInt64 i = 0; i < num_rows; ++i) { ExternalModelInfo info; - readStringBinary(info.model_path, buf); - readStringBinary(info.model_type, buf); + readStringBinary(info.model_path, *buf); + readStringBinary(info.model_type, *buf); UInt64 t; - readIntBinary(t, buf); + readIntBinary(t, *buf); info.loading_start_time = std::chrono::system_clock::from_time_t(t); - readIntBinary(t, buf); + readIntBinary(t, *buf); info.loading_duration = std::chrono::milliseconds(t); result.push_back(info); @@ -116,17 +120,19 @@ void CatBoostLibraryBridgeHelper::removeModel() assert(model_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_REMOVEMODEL_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this](std::ostream & os) - { - os << "model_path=" << escapeForFileName(*model_path); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEMODEL_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this](std::ostream & os) + { + os << "model_path=" << escapeForFileName(*model_path); + }) + .create(credentials); String result; - readStringBinary(result, buf); + readStringBinary(result, *buf); assert(result == "1"); } @@ -134,14 +140,14 @@ void CatBoostLibraryBridgeHelper::removeAllModels() { startBridgeSync(); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [](std::ostream &){}, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .create(credentials); String result; - readStringBinary(result, buf); + readStringBinary(result, *buf); assert(result == "1"); } @@ -151,18 +157,20 @@ size_t CatBoostLibraryBridgeHelper::getTreeCount() assert(model_path && library_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_GETTREECOUNT_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this](std::ostream & os) - { - os << "library_path=" << escapeForFileName(*library_path) << "&"; - os << "model_path=" << escapeForFileName(*model_path); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_GETTREECOUNT_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this](std::ostream & os) + { + os << "library_path=" << escapeForFileName(*library_path) << "&"; + os << "model_path=" << escapeForFileName(*model_path); + }) + .create(credentials); size_t result; - readIntBinary(result, buf); + readIntBinary(result, *buf); return result; } @@ -177,17 +185,19 @@ ColumnPtr CatBoostLibraryBridgeHelper::evaluate(const ColumnsWithTypeAndName & c assert(model_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_LIB_EVALUATE_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this, serialized = string_write_buf.str()](std::ostream & os) - { - os << "model_path=" << escapeForFileName(*model_path) << "&"; - os << "data=" << escapeForFileName(serialized); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIB_EVALUATE_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this, serialized = string_write_buf.str()](std::ostream & os) + { + os << "model_path=" << escapeForFileName(*model_path) << "&"; + os << "data=" << escapeForFileName(serialized); + }) + .create(credentials); - NativeReader deserializer(buf, /*server_revision*/ 0); + NativeReader deserializer(*buf, /*server_revision*/ 0); Block block_read = deserializer.read(); return block_read.getColumns()[0]; diff --git a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp index fcb8ebd1f22..a37d2bd396f 100644 --- a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp +++ b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp @@ -71,8 +71,12 @@ bool ExternalDictionaryLibraryBridgeHelper::bridgeHandShake() String result; try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials); - readString(result, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(http_timeouts) + .create(credentials); + + readString(result, *buf); } catch (...) { @@ -247,30 +251,28 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadKeys(const Block & requ bool ExternalDictionaryLibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) const { - ReadWriteBufferFromHTTP buf( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - std::move(out_stream_callback), - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback(std::move(out_stream_callback)) + .create(credentials); bool res; - readBoolText(res, buf); + readBoolText(res, *buf); return res; } QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) { - auto read_buf_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - std::move(out_stream_callback), - http_timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - getContext()->getReadSettings(), - HTTPHeaderEntries{}); + auto read_buf_ptr = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(http_timeouts) + .withOutCallback(std::move(out_stream_callback)) + .create(credentials); auto source = FormatFactory::instance().getInput(ExternalDictionaryLibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); source->addBuffer(std::move(read_buf_ptr)); diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index d208b8ddab0..838be42357a 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -97,8 +97,12 @@ protected: { try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, getHTTPTimeouts(), credentials); - return checkString(PING_OK_ANSWER, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); + + return checkString(PING_OK_ANSWER, *buf); } catch (...) { @@ -198,10 +202,14 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); - bool res; - readBoolText(res, buf); + bool res = false; + readBoolText(res, *buf); is_schema_allowed = res; } @@ -220,10 +228,14 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); std::string character; - readStringBinary(character, buf); + readStringBinary(character, *buf); if (character.length() > 1) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Failed to parse quoting style from '{}' for service {}", character, BridgeHelperMixin::serviceAlias()); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 4848f4c7ee5..a11a1243957 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -153,6 +153,12 @@ void Connection::connect(const ConnectionTimeouts & timeouts) current_resolved_address = *it; break; } + catch (DB::NetException &) + { + if (++it == addresses.end()) + throw; + continue; + } catch (Poco::Net::NetException &) { if (++it == addresses.end()) @@ -199,6 +205,16 @@ void Connection::connect(const ConnectionTimeouts & timeouts) LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); } + catch (DB::NetException & e) + { + disconnect(); + + /// Remove this possible stale entry from cache + DNSResolver::instance().removeHostFromCache(host); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); + } catch (Poco::Net::NetException & e) { disconnect(); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82da4c4bbad..f43481f665b 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -275,6 +275,18 @@ M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") + \ + M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ + M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ + \ + M(DiskConnectionsStored, "Total count of sessions stored in the session pool for disks") \ + M(DiskConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for disks") \ + \ + M(HTTPConnectionsStored, "Total count of sessions stored in the session pool for http hosts") \ + M(HTTPConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for http hosts") \ + \ + M(AddressesActive, "Total count of addresses which are used for creation connections with connection pools") \ + #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 5b5f5369d5e..e36e1483da8 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -1,6 +1,7 @@ #include "DNSResolver.h" #include #include +#include #include #include #include @@ -108,7 +109,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host) if (addresses.empty()) { ProfileEvents::increment(ProfileEvents::DNSError); - throw Exception(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host); + throw DB::NetException(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host); } return addresses; diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp new file mode 100644 index 00000000000..18ffef34091 --- /dev/null +++ b/src/Common/HTTPConnectionPool.cpp @@ -0,0 +1,865 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include + +#include "config.h" + +#if USE_SSL +#include +#endif + + +namespace ProfileEvents +{ + extern const Event StorageConnectionsCreated; + extern const Event StorageConnectionsReused; + extern const Event StorageConnectionsReset; + extern const Event StorageConnectionsPreserved; + extern const Event StorageConnectionsExpired; + extern const Event StorageConnectionsErrors; + extern const Event StorageConnectionsElapsedMicroseconds; + + extern const Event DiskConnectionsCreated; + extern const Event DiskConnectionsReused; + extern const Event DiskConnectionsReset; + extern const Event DiskConnectionsPreserved; + extern const Event DiskConnectionsExpired; + extern const Event DiskConnectionsErrors; + extern const Event DiskConnectionsElapsedMicroseconds; + + extern const Event HTTPConnectionsCreated; + extern const Event HTTPConnectionsReused; + extern const Event HTTPConnectionsReset; + extern const Event HTTPConnectionsPreserved; + extern const Event HTTPConnectionsExpired; + extern const Event HTTPConnectionsErrors; + extern const Event HTTPConnectionsElapsedMicroseconds; +} + + +namespace CurrentMetrics +{ + extern const Metric StorageConnectionsStored; + extern const Metric StorageConnectionsTotal; + + extern const Metric DiskConnectionsStored; + extern const Metric DiskConnectionsTotal; + + extern const Metric HTTPConnectionsStored; + extern const Metric HTTPConnectionsTotal; +} + +namespace +{ + Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) + { + Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; + + poco_proxy_config.host = proxy_configuration.host; + poco_proxy_config.port = proxy_configuration.port; + poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); + poco_proxy_config.tunnel = proxy_configuration.tunneling; + poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); + + return poco_proxy_config; + } + + + size_t roundUp(size_t x, size_t rounding) + { + chassert(rounding > 0); + return (x + (rounding - 1)) / rounding * rounding; + } + + + Poco::Timespan divide(const Poco::Timespan span, int divisor) + { + return Poco::Timespan(Poco::Timestamp::TimeDiff(span.totalMicroseconds() / divisor)); + } +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; + extern const int UNSUPPORTED_URI_SCHEME; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForStorageConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::StorageConnectionsCreated, + .reused = ProfileEvents::StorageConnectionsReused, + .reset = ProfileEvents::StorageConnectionsReset, + .preserved = ProfileEvents::StorageConnectionsPreserved, + .expired = ProfileEvents::StorageConnectionsExpired, + .errors = ProfileEvents::StorageConnectionsErrors, + .elapsed_microseconds = ProfileEvents::StorageConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::StorageConnectionsStored, + .active_count = CurrentMetrics::StorageConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForDiskConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::DiskConnectionsCreated, + .reused = ProfileEvents::DiskConnectionsReused, + .reset = ProfileEvents::DiskConnectionsReset, + .preserved = ProfileEvents::DiskConnectionsPreserved, + .expired = ProfileEvents::DiskConnectionsExpired, + .errors = ProfileEvents::DiskConnectionsErrors, + .elapsed_microseconds = ProfileEvents::DiskConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::DiskConnectionsStored, + .active_count = CurrentMetrics::DiskConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForHTTPConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::HTTPConnectionsCreated, + .reused = ProfileEvents::HTTPConnectionsReused, + .reset = ProfileEvents::HTTPConnectionsReset, + .preserved = ProfileEvents::HTTPConnectionsPreserved, + .expired = ProfileEvents::HTTPConnectionsExpired, + .errors = ProfileEvents::HTTPConnectionsErrors, + .elapsed_microseconds = ProfileEvents::HTTPConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::HTTPConnectionsStored, + .active_count = CurrentMetrics::HTTPConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getConnectionPoolMetrics(HTTPConnectionGroupType type) +{ + switch (type) + { + case HTTPConnectionGroupType::STORAGE: + return getMetricsForStorageConnectionPool(); + case HTTPConnectionGroupType::DISK: + return getMetricsForDiskConnectionPool(); + case HTTPConnectionGroupType::HTTP: + return getMetricsForHTTPConnectionPool(); + } +} + + +class ConnectionGroup +{ +public: + using Ptr = std::shared_ptr; + + explicit ConnectionGroup(HTTPConnectionGroupType type_) : type(type_), metrics(getConnectionPoolMetrics(type_)) { } + + void setLimits(HTTPConnectionPools::Limits limits_) + { + std::lock_guard lock(mutex); + limits = std::move(limits_); + mute_warning_until = 0; + } + + bool isSoftLimitReached() const + { + std::lock_guard lock(mutex); + return total_connections_in_group >= limits.soft_limit; + } + + bool isStoreLimitReached() const + { + std::lock_guard lock(mutex); + return total_connections_in_group >= limits.store_limit; + } + + void atConnectionCreate() + { + std::lock_guard lock(mutex); + + ++total_connections_in_group; + + if (total_connections_in_group >= limits.warning_limit && total_connections_in_group >= mute_warning_until) + { + LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}", type, total_connections_in_group, limits.warning_limit); + mute_warning_until = roundUp(total_connections_in_group, limits.warning_step); + } + } + + void atConnectionDestroy() + { + std::lock_guard lock(mutex); + + --total_connections_in_group; + + const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 10 : 1; + if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit) + { + LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group); + mute_warning_until = 0; + } + } + + HTTPConnectionGroupType getType() const { return type; } + + const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const { return metrics; } + +private: + const HTTPConnectionGroupType type; + const IHTTPConnectionPoolForEndpoint::Metrics metrics; + + LoggerPtr log = getLogger("ConnectionGroup"); + + mutable std::mutex mutex; + HTTPConnectionPools::Limits limits TSA_GUARDED_BY(mutex) = HTTPConnectionPools::Limits(); + size_t total_connections_in_group TSA_GUARDED_BY(mutex) = 0; + size_t mute_warning_until TSA_GUARDED_BY(mutex) = 0; +}; + + +class IExtendedPool : public IHTTPConnectionPoolForEndpoint +{ +public: + using Ptr = std::shared_ptr; + + virtual HTTPConnectionGroupType getGroupType() const = 0; + virtual size_t wipeExpired() = 0; +}; + + +// EndpointConnectionPool manage connections to the endpoint +// Features: +// - it uses HostResolver for address selecting. See Common/HostResolver.h for more info. +// - it minimizes number of `Session::connect()`/`Session::reconnect()` calls +// - stores only connected and ready to use sessions +// - connection could be reused even when limits are reached +// - soft limit, warn limit, store limit +// - `Session::reconnect()` uses the pool as well +// - comprehensive sensors +// - session is reused according its inner state, automatically + + +template +class EndpointConnectionPool : public std::enable_shared_from_this>, public IExtendedPool +{ +private: + friend class HTTPConnectionPools; + + using WeakPtr = std::weak_ptr>; + + class PooledConnection : public Session + { + public: + using Ptr = std::shared_ptr; + + void reconnect() override + { + ProfileEvents::increment(metrics.reset); + Session::close(); + + if (auto lock = pool.lock()) + { + auto timeouts = getTimeouts(*this); + auto new_connection = lock->getConnection(timeouts); + Session::assign(*new_connection); + } + else + { + auto timer = CurrentThread::getProfileEvents().timer(metrics.elapsed_microseconds); + Session::reconnect(); + ProfileEvents::increment(metrics.created); + } + } + + String getTarget() const + { + if (!Session::getProxyConfig().host.empty()) + return fmt::format("{} over proxy {}", Session::getHost(), Session::getProxyConfig().host); + return Session::getHost(); + } + + void flushRequest() override + { + if (bool(request_stream)) + { + request_stream->flush(); + + if (auto * fixed_steam = dynamic_cast(request_stream)) + { + request_stream_completed = fixed_steam->isComplete(); + } + else if (auto * chunked_steam = dynamic_cast(request_stream)) + { + chunked_steam->rdbuf()->close(); + request_stream_completed = chunked_steam->isComplete(); + } + else if (auto * http_stream = dynamic_cast(request_stream)) + { + request_stream_completed = http_stream->isComplete(); + } + else + { + request_stream_completed = false; + } + } + request_stream = nullptr; + + Session::flushRequest(); + } + + std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override + { + std::ostream & result = Session::sendRequest(request); + result.exceptions(std::ios::badbit); + + request_stream = &result; + request_stream_completed = false; + + response_stream = nullptr; + response_stream_completed = true; + + return result; + } + + std::istream & receiveResponse(Poco::Net::HTTPResponse & response) override + { + std::istream & result = Session::receiveResponse(response); + result.exceptions(std::ios::badbit); + + response_stream = &result; + response_stream_completed = false; + + return result; + } + + void reset() override + { + request_stream = nullptr; + request_stream_completed = false; + + response_stream = nullptr; + response_stream_completed = false; + + Session::reset(); + } + + ~PooledConnection() override + { + if (bool(response_stream)) + { + if (auto * fixed_steam = dynamic_cast(response_stream)) + { + response_stream_completed = fixed_steam->isComplete(); + } + else if (auto * chunked_steam = dynamic_cast(response_stream)) + { + response_stream_completed = chunked_steam->isComplete(); + } + else if (auto * http_stream = dynamic_cast(response_stream)) + { + response_stream_completed = http_stream->isComplete(); + } + else + { + response_stream_completed = false; + } + } + response_stream = nullptr; + + if (auto lock = pool.lock()) + lock->atConnectionDestroy(*this); + else + ProfileEvents::increment(metrics.reset); + + CurrentMetrics::sub(metrics.active_count); + } + + private: + friend class EndpointConnectionPool; + + template + explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) + : Session(args...), pool(std::move(pool_)), metrics(std::move(metrics_)) + { + CurrentMetrics::add(metrics.active_count); + } + + template + static Ptr create(Args &&... args) + { + /// Pool is global, we shouldn't attribute this memory to query/user. + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + + struct make_shared_enabler : public PooledConnection + { + explicit make_shared_enabler(Args &&... args) : PooledConnection(std::forward(args)...) { } + }; + return std::make_shared(std::forward(args)...); + } + + void doConnect() + { + Session::reconnect(); + } + + bool isCompleted() const + { + return request_stream_completed && response_stream_completed; + } + + WeakPtr pool; + IHTTPConnectionPoolForEndpoint::Metrics metrics; + + Poco::Logger * log = &Poco::Logger::get("PooledConnection"); + + std::ostream * request_stream = nullptr; + std::istream * response_stream = nullptr; + + bool request_stream_completed = true; + bool response_stream_completed = true; + }; + + using Connection = PooledConnection; + using ConnectionPtr = PooledConnection::Ptr; + + struct GreaterByLastRequest + { + static bool operator()(const ConnectionPtr & l, const ConnectionPtr & r) + { + return l->getLastRequest() + l->getKeepAliveTimeout() > r->getLastRequest() + r->getKeepAliveTimeout(); + } + }; + + using ConnectionsMinHeap = std::priority_queue, GreaterByLastRequest>; + +public: + template + static Ptr create(Args &&... args) + { + struct make_shared_enabler : public EndpointConnectionPool + { + explicit make_shared_enabler(Args &&... args) : EndpointConnectionPool(std::forward(args)...) { } + }; + return std::make_shared(std::forward(args)...); + } + + ~EndpointConnectionPool() override + { + CurrentMetrics::sub(group->getMetrics().stored_count, stored_connections.size()); + } + + String getTarget() const + { + if (!proxy_configuration.isEmpty()) + return fmt::format("{} over proxy {}", host, proxy_configuration.host); + return host; + } + + IHTTPConnectionPoolForEndpoint::ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) override + { + Poco::Timestamp now; + std::vector expired_connections; + + SCOPE_EXIT({ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + expired_connections.clear(); + }); + + { + std::lock_guard lock(mutex); + + wipeExpiredImpl(expired_connections, now); + + if (!stored_connections.empty()) + { + auto it = stored_connections.top(); + stored_connections.pop(); + + setTimeouts(*it, timeouts); + + ProfileEvents::increment(getMetrics().reused, 1); + CurrentMetrics::sub(getMetrics().stored_count, 1); + + return it; + } + } + + return prepareNewConnection(timeouts); + } + + const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const override + { + return group->getMetrics(); + } + + HTTPConnectionGroupType getGroupType() const override + { + return group->getType(); + } + + size_t wipeExpired() override + { + Poco::Timestamp now; + std::vector expired_connections; + + SCOPE_EXIT({ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + expired_connections.clear(); + }); + + std::lock_guard lock(mutex); + return wipeExpiredImpl(expired_connections, now); + } + + size_t wipeExpiredImpl(std::vector & expired_connections, Poco::Timestamp now) TSA_REQUIRES(mutex) + { + while (!stored_connections.empty()) + { + auto connection = stored_connections.top(); + + if (!isExpired(now, connection)) + return stored_connections.size(); + + stored_connections.pop(); + expired_connections.push_back(connection); + } + + CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size()); + ProfileEvents::increment(getMetrics().expired, expired_connections.size()); + + return stored_connections.size(); + } + +private: + EndpointConnectionPool(ConnectionGroup::Ptr group_, String host_, UInt16 port_, bool https_, ProxyConfiguration proxy_configuration_) + : host(std::move(host_)) + , port(port_) + , https(https_) + , proxy_configuration(std::move(proxy_configuration_)) + , group(group_) + { + } + + WeakPtr getWeakFromThis() { return EndpointConnectionPool::weak_from_this(); } + + bool isExpired(Poco::Timestamp & now, ConnectionPtr connection) + { + if (group->isSoftLimitReached()) + return now > (connection->getLastRequest() + divide(connection->getKeepAliveTimeout(), 10)); + return now > connection->getLastRequest() + connection->getKeepAliveTimeout(); + } + + ConnectionPtr allocateNewConnection() + { + ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), getMetrics(), host, port); + connection->setKeepAlive(true); + + if (!proxy_configuration.isEmpty()) + { + connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); + } + + group->atConnectionCreate(); + + return connection; + } + + ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts) + { + auto address = HostResolversPool::instance().getResolver(host)->resolve(); + + auto session = allocateNewConnection(); + + setTimeouts(*session, timeouts); + session->setResolvedHost(*address); + + try + { + auto timer = CurrentThread::getProfileEvents().timer(getMetrics().elapsed_microseconds); + session->doConnect(); + } + catch (...) + { + address.setFail(); + ProfileEvents::increment(getMetrics().errors); + session->reset(); + throw; + } + + ProfileEvents::increment(getMetrics().created); + return session; + } + + void atConnectionDestroy(PooledConnection & connection) + { + group->atConnectionDestroy(); + + if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() + || group->isStoreLimitReached()) + { + ProfileEvents::increment(getMetrics().reset, 1); + return; + } + + auto connection_to_store = allocateNewConnection(); + connection_to_store->assign(connection); + + CurrentMetrics::add(getMetrics().stored_count, 1); + ProfileEvents::increment(getMetrics().preserved, 1); + + { + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + std::lock_guard lock(mutex); + stored_connections.push(connection_to_store); + } + } + + + const std::string host; + const UInt16 port; + const bool https; + const ProxyConfiguration proxy_configuration; + const ConnectionGroup::Ptr group; + + std::mutex mutex; + ConnectionsMinHeap stored_connections TSA_GUARDED_BY(mutex); +}; + +struct EndpointPoolKey +{ + HTTPConnectionGroupType connection_group; + String target_host; + UInt16 target_port; + bool is_target_https; + ProxyConfiguration proxy_config; + + bool operator==(const EndpointPoolKey & rhs) const + { + return std::tie( + connection_group, + target_host, + target_port, + is_target_https, + proxy_config.host, + proxy_config.port, + proxy_config.protocol, + proxy_config.tunneling, + proxy_config.original_request_protocol) + == std::tie( + rhs.connection_group, + rhs.target_host, + rhs.target_port, + rhs.is_target_https, + rhs.proxy_config.host, + rhs.proxy_config.port, + rhs.proxy_config.protocol, + rhs.proxy_config.tunneling, + rhs.proxy_config.original_request_protocol); + } +}; + +struct Hasher +{ + size_t operator()(const EndpointPoolKey & k) const + { + SipHash s; + s.update(k.connection_group); + s.update(k.target_host); + s.update(k.target_port); + s.update(k.is_target_https); + s.update(k.proxy_config.host); + s.update(k.proxy_config.port); + s.update(k.proxy_config.protocol); + s.update(k.proxy_config.tunneling); + s.update(k.proxy_config.original_request_protocol); + return s.get64(); + } +}; + +IExtendedPool::Ptr +createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, bool secure, ProxyConfiguration proxy_configuration) +{ + if (secure) + { +#if USE_SSL + return EndpointConnectionPool::create( + group, std::move(host), port, secure, std::move(proxy_configuration)); +#else + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "Inter-server secret support is disabled, because ClickHouse was built without SSL library"); +#endif + } + else + { + return EndpointConnectionPool::create( + group, std::move(host), port, secure, std::move(proxy_configuration)); + } +} + +class HTTPConnectionPools::Impl +{ +private: + const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 5 * 60; + const Poco::Timespan wipe_timeout = Poco::Timespan(DEFAULT_WIPE_TIMEOUT_SECONDS, 0); + + ConnectionGroup::Ptr disk_group = std::make_shared(HTTPConnectionGroupType::DISK); + ConnectionGroup::Ptr storage_group = std::make_shared(HTTPConnectionGroupType::STORAGE); + ConnectionGroup::Ptr http_group = std::make_shared(HTTPConnectionGroupType::HTTP); + + + /// If multiple mutexes are held simultaneously, + /// they should be locked in this order: + /// HTTPConnectionPools::mutex, then EndpointConnectionPool::mutex, then ConnectionGroup::mutex. + std::mutex mutex; + + std::unordered_map endpoints_pool TSA_GUARDED_BY(mutex); + Poco::Timestamp last_wipe_time TSA_GUARDED_BY(mutex); + +public: + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + Poco::Timestamp now; + + std::lock_guard lock(mutex); + + if (now - last_wipe_time > wipe_timeout) + { + wipeExpired(); + last_wipe_time = now; + } + + return getPoolImpl(type, uri, proxy_configuration); + } + + void setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http) + { + disk_group->setLimits(std::move(disk)); + storage_group->setLimits(std::move(storage)); + http_group->setLimits(std::move(http)); + } + + void dropCache() + { + std::lock_guard lock(mutex); + endpoints_pool.clear(); + } + +protected: + ConnectionGroup::Ptr & getGroup(HTTPConnectionGroupType type) + { + switch (type) + { + case HTTPConnectionGroupType::DISK: + return disk_group; + case HTTPConnectionGroupType::STORAGE: + return storage_group; + case HTTPConnectionGroupType::HTTP: + return http_group; + } + } + + IExtendedPool::Ptr getPoolImpl(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + TSA_REQUIRES(mutex) + { + auto [host, port, secure] = getHostPortSecure(uri, proxy_configuration); + auto key = EndpointPoolKey{type, host, port, secure, proxy_configuration}; + + auto it = endpoints_pool.find(key); + if (it != endpoints_pool.end()) + return it->second; + + it = endpoints_pool.emplace(key, createConnectionPool(getGroup(type), std::move(host), port, secure, proxy_configuration)).first; + + return it->second; + } + + void wipeExpired() TSA_REQUIRES(mutex) + { + std::vector keys_to_drop; + + for (auto & [key, pool] : endpoints_pool) + { + auto left_connections = pool->wipeExpired(); + if (left_connections == 0 && pool->getGroupType() != HTTPConnectionGroupType::DISK) + keys_to_drop.push_back(key); + } + + for (const auto & key : keys_to_drop) + endpoints_pool.erase(key); + } + + static bool useSecureConnection(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + if (uri.getScheme() == "http") + return false; + + if (uri.getScheme() != "https") + throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); + + if (!proxy_configuration.isEmpty()) + { + if (ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol && !proxy_configuration.tunneling) + { + // If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request. + // The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server. + // There is a weak link in the security, but that's what the user opted for. + return false; + } + } + + return true; + } + + static std::tuple getHostPortSecure(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + return std::make_tuple(uri.getHost(), uri.getPort(), useSecureConnection(uri, proxy_configuration)); + } +}; + +HTTPConnectionPools::HTTPConnectionPools() + : impl(std::make_unique()) +{ +} + +HTTPConnectionPools & HTTPConnectionPools::instance() +{ + static HTTPConnectionPools instance; + return instance; +} + +void HTTPConnectionPools::setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http) +{ + impl->setLimits(std::move(disk), std::move(storage), std::move(http)); +} + +void HTTPConnectionPools::dropCache() +{ + impl->dropCache(); +} + +IHTTPConnectionPoolForEndpoint::Ptr +HTTPConnectionPools::getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) +{ + return impl->getPool(type, uri, proxy_configuration); +} +} diff --git a/src/Common/HTTPConnectionPool.h b/src/Common/HTTPConnectionPool.h new file mode 100644 index 00000000000..7fd0847f5a7 --- /dev/null +++ b/src/Common/HTTPConnectionPool.h @@ -0,0 +1,91 @@ +#pragma once + +#include + +#include +#include +#include +#include + +#include + +#include +#include + +#include +#include + +namespace DB +{ + +class IHTTPConnectionPoolForEndpoint +{ +public: + struct Metrics + { + const ProfileEvents::Event created = ProfileEvents::end(); + const ProfileEvents::Event reused = ProfileEvents::end(); + const ProfileEvents::Event reset = ProfileEvents::end(); + const ProfileEvents::Event preserved = ProfileEvents::end(); + const ProfileEvents::Event expired = ProfileEvents::end(); + const ProfileEvents::Event errors = ProfileEvents::end(); + const ProfileEvents::Event elapsed_microseconds = ProfileEvents::end(); + + const CurrentMetrics::Metric stored_count = CurrentMetrics::end(); + const CurrentMetrics::Metric active_count = CurrentMetrics::end(); + }; + + using Ptr = std::shared_ptr; + using Connection = Poco::Net::HTTPClientSession; + using ConnectionPtr = std::shared_ptr; + + /// can throw Poco::Net::Exception, DB::NetException, DB::Exception + virtual ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) = 0; + virtual const Metrics & getMetrics() const = 0; + virtual ~IHTTPConnectionPoolForEndpoint() = default; + +protected: + IHTTPConnectionPoolForEndpoint() = default; + + IHTTPConnectionPoolForEndpoint(const IHTTPConnectionPoolForEndpoint &) = delete; + IHTTPConnectionPoolForEndpoint & operator=(const IHTTPConnectionPoolForEndpoint &) = delete; +}; + +enum class HTTPConnectionGroupType +{ + DISK, + STORAGE, + HTTP, +}; + +class HTTPConnectionPools +{ +public: + struct Limits + { + size_t soft_limit = 100; + size_t warning_limit = 1000; + size_t store_limit = 10000; + + static constexpr size_t warning_step = 100; + }; + +private: + HTTPConnectionPools(); + HTTPConnectionPools(const HTTPConnectionPools &) = delete; + HTTPConnectionPools & operator=(const HTTPConnectionPools &) = delete; + +public: + static HTTPConnectionPools & instance(); + + void setLimits(Limits disk, Limits storage, Limits http); + void dropCache(); + + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration); + +private: + class Impl; + std::unique_ptr impl; +}; + +} diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp new file mode 100644 index 00000000000..f6cc9c919ba --- /dev/null +++ b/src/Common/HostResolvePool.cpp @@ -0,0 +1,293 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace ProfileEvents +{ + extern const Event AddressesDiscovered; + extern const Event AddressesExpired; + extern const Event AddressesFailScored; +} + +namespace CurrentMetrics +{ + extern const Metric AddressesActive; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DNS_ERROR; +} + +HostResolverMetrics HostResolver::getMetrics() +{ + return HostResolverMetrics{ + .discovered = ProfileEvents::AddressesDiscovered, + .expired = ProfileEvents::AddressesExpired, + .failed = ProfileEvents::AddressesFailScored, + .active_count = CurrentMetrics::AddressesActive, + }; +} + +HostResolver::WeakPtr HostResolver::getWeakFromThis() +{ + return weak_from_this(); +} + +HostResolver::HostResolver(String host_, Poco::Timespan history_) + : host(std::move(host_)) + , history(history_) + , resolve_function([](const String & host_to_resolve) { return DNSResolver::instance().resolveHostAll(host_to_resolve); }) +{ + update(); +} + +HostResolver::HostResolver( + ResolveFunction && resolve_function_, String host_, Poco::Timespan history_) + : host(std::move(host_)), history(history_), resolve_function(std::move(resolve_function_)) +{ + update(); +} + +HostResolver::~HostResolver() +{ + std::lock_guard lock(mutex); + CurrentMetrics::sub(metrics.active_count, records.size()); + records.clear(); +} + +void HostResolver::Entry::setFail() +{ + fail = true; + + if (auto lock = pool.lock()) + lock->setFail(address); +} + +HostResolver::Entry::~Entry() +{ + if (!fail) + { + if (auto lock = pool.lock()) + lock->setSuccess(address); + } +} + +void HostResolver::update() +{ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + + auto next_gen = resolve_function(host); + if (next_gen.empty()) + throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host); + + std::sort(next_gen.begin(), next_gen.end()); + + Poco::Timestamp now; + + std::lock_guard lock(mutex); + updateImpl(now, next_gen); +} + +void HostResolver::reset() +{ + std::lock_guard lock(mutex); + + CurrentMetrics::sub(metrics.active_count, records.size()); + records.clear(); +} + +void HostResolver::updateWeights() +{ + updateWeightsImpl(); + + if (getTotalWeight() == 0 && !records.empty()) + { + for (auto & rec : records) + rec.failed = false; + + updateWeightsImpl(); + } + + chassert((getTotalWeight() > 0 && !records.empty()) || records.empty()); + random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); +} + +HostResolver::Entry HostResolver::resolve() +{ + if (isUpdateNeeded()) + update(); + + std::lock_guard lock(mutex); + return Entry(*this, selectBest()); +} + +void HostResolver::setSuccess(const Poco::Net::IPAddress & address) +{ + std::lock_guard lock(mutex); + + auto it = find(address); + if (it == records.end()) + return; + + auto old_weight = it->getWeight(); + ++it->usage; + auto new_weight = it->getWeight(); + + if (old_weight != new_weight) + updateWeights(); +} + +void HostResolver::setFail(const Poco::Net::IPAddress & address) +{ + Poco::Timestamp now; + + { + std::lock_guard lock(mutex); + + auto it = find(address); + if (it == records.end()) + return; + + it->failed = true; + it->fail_time = now; + } + + ProfileEvents::increment(metrics.failed); + update(); +} + +Poco::Net::IPAddress HostResolver::selectBest() +{ + chassert(!records.empty()); + size_t weight = random_weight_picker(thread_local_rng); + auto it = std::partition_point(records.begin(), records.end(), [&](const Record & rec) { return rec.weight_prefix_sum <= weight; }); + chassert(it != records.end()); + return it->address; +} + +HostResolver::Records::iterator HostResolver::find(const Poco::Net::IPAddress & addr) TSA_REQUIRES(mutex) +{ + return std::lower_bound( + records.begin(), records.end(), addr, [](const Record & rec, const Poco::Net::IPAddress & value) { return rec.address < value; }); +} + +bool HostResolver::isUpdateNeeded() +{ + Poco::Timestamp now; + + std::lock_guard lock(mutex); + return last_resolve_time + history < now || records.empty(); +} + +void HostResolver::updateImpl(Poco::Timestamp now, std::vector & next_gen) + TSA_REQUIRES(mutex) +{ + const auto last_effective_resolve = now - history; + + Records merged; + merged.reserve(records.size() + next_gen.size()); + + auto it_before = records.begin(); + auto it_next = next_gen.begin(); + + while (it_before != records.end() || it_next != next_gen.end()) + { + if (it_next == next_gen.end() || (it_before != records.end() && it_before->address < *it_next)) + { + if (it_before->resolve_time >= last_effective_resolve) + merged.push_back(*it_before); + else + { + CurrentMetrics::sub(metrics.active_count, 1); + ProfileEvents::increment(metrics.expired, 1); + } + ++it_before; + } + else if (it_before == records.end() || (it_next != next_gen.end() && *it_next < it_before->address)) + { + CurrentMetrics::add(metrics.active_count, 1); + ProfileEvents::increment(metrics.discovered, 1); + merged.push_back(Record(*it_next, now)); + ++it_next; + } + else + { + merged.push_back(*it_before); + merged.back().resolve_time = now; + + ++it_before; + ++it_next; + } + } + + for (auto & rec : merged) + if (rec.failed && rec.fail_time < last_effective_resolve) + rec.failed = false; + + chassert(std::is_sorted(merged.begin(), merged.end())); + + last_resolve_time = now; + records.swap(merged); + + if (records.empty()) + throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host); + + updateWeights(); +} + +size_t HostResolver::getTotalWeight() const +{ + if (records.empty()) + return 0; + return records.back().weight_prefix_sum; +} + + +void HostResolver::updateWeightsImpl() +{ + size_t total_weight_next = 0; + + for (auto & rec: records) + { + total_weight_next += rec.getWeight(); + rec.weight_prefix_sum = total_weight_next; + } +} + +HostResolversPool & HostResolversPool::instance() +{ + static HostResolversPool instance; + return instance; +} + +void HostResolversPool::dropCache() +{ + std::lock_guard lock(mutex); + host_pools.clear(); +} + +HostResolver::Ptr HostResolversPool::getResolver(const String & host) +{ + std::lock_guard lock(mutex); + + auto it = host_pools.find(host); + if (it != host_pools.end()) + return it->second; + + it = host_pools.emplace(host, HostResolver::create(host)).first; + + return it->second; +} + +} diff --git a/src/Common/HostResolvePool.h b/src/Common/HostResolvePool.h new file mode 100644 index 00000000000..2a31cec3b2d --- /dev/null +++ b/src/Common/HostResolvePool.h @@ -0,0 +1,218 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include + +#include +#include + +// That class resolves host into multiply addresses +// Features: +// - balance address usage. +// `selectBest()` chooses the address by random with weights. +// The more ip is used the lesser weight it has. When new address is happened, it takes more weight. +// But still not all requests are assigned to the new address. +// - join resolve results +// In case when host is resolved into different set of addresses, this class join all that addresses and use them. +// An address expires after `history_` time. +// - failed address pessimization +// If an address marked with `setFail()` it is marked as faulty. Such address won't be selected until either +// a) it still occurs in resolve set after `history_` time or b) all other addresses are pessimized as well. +// - resolve schedule +// Addresses are resolved through `DB::DNSResolver::instance()`. +// Usually it does not happen more often than once in `history_` time. +// But also new resolve performed each `setFail()` call. + +namespace DB +{ + +struct HostResolverMetrics +{ + const ProfileEvents::Event discovered = ProfileEvents::end(); + const ProfileEvents::Event expired = ProfileEvents::end(); + const ProfileEvents::Event failed = ProfileEvents::end(); + + const CurrentMetrics::Metric active_count = CurrentMetrics::end(); +}; + +constexpr size_t DEFAULT_RESOLVE_TIME_HISTORY_SECONDS = 2*60; + + +class HostResolver : public std::enable_shared_from_this +{ +private: + using WeakPtr = std::weak_ptr; + +public: + using Ptr = std::shared_ptr; + + template + static Ptr create(Args&&... args) + { + struct make_shared_enabler : public HostResolver + { + explicit make_shared_enabler(Args&&... args) : HostResolver(std::forward(args)...) {} + }; + return std::make_shared(std::forward(args)...); + } + + virtual ~HostResolver(); + + class Entry + { + public: + explicit Entry(Entry && entry) = default; + explicit Entry(Entry & entry) = delete; + + // no access as r-value + const String * operator->() && = delete; + const String * operator->() const && = delete; + const String & operator*() && = delete; + const String & operator*() const && = delete; + + const String * operator->() & { return &resolved_host; } + const String * operator->() const & { return &resolved_host; } + const String & operator*() & { return resolved_host; } + const String & operator*() const & { return resolved_host; } + + void setFail(); + ~Entry(); + + private: + friend class HostResolver; + + Entry(HostResolver & pool_, Poco::Net::IPAddress address_) + : pool(pool_.getWeakFromThis()) + , address(std::move(address_)) + , resolved_host(address.toString()) + { } + + HostResolver::WeakPtr pool; + const Poco::Net::IPAddress address; + const String resolved_host; + + bool fail = false; + }; + + /// can throw NetException(ErrorCodes::DNS_ERROR, ...), Exception(ErrorCodes::BAD_ARGUMENTS, ...) + Entry resolve(); + void update(); + void reset(); + + static HostResolverMetrics getMetrics(); + +protected: + explicit HostResolver( + String host_, + Poco::Timespan history_ = Poco::Timespan(DEFAULT_RESOLVE_TIME_HISTORY_SECONDS, 0)); + + using ResolveFunction = std::function (const String & host)>; + HostResolver(ResolveFunction && resolve_function_, + String host_, + Poco::Timespan history_); + + friend class Entry; + WeakPtr getWeakFromThis(); + + void setSuccess(const Poco::Net::IPAddress & address); + void setFail(const Poco::Net::IPAddress & address); + + struct Record + { + Record(Poco::Net::IPAddress address_, Poco::Timestamp resolve_time_) + : address(std::move(address_)) + , resolve_time(resolve_time_) + {} + + explicit Record(Record && rec) = default; + Record& operator=(Record && s) = default; + + explicit Record(const Record & rec) = default; + Record& operator=(const Record & s) = default; + + Poco::Net::IPAddress address; + Poco::Timestamp resolve_time; + size_t usage = 0; + bool failed = false; + Poco::Timestamp fail_time = 0; + + size_t weight_prefix_sum; + + bool operator <(const Record & r) const + { + return address < r.address; + } + + size_t getWeight() const + { + if (failed) + return 0; + + /// There is no goal to make usage's distribution ideally even + /// The goal is to chose more often new address, but still use old addresses as well + /// when all addresses have usage counter greater than 10000, + /// no more corrections are needed, just random choice is ok + if (usage > 10000) + return 1; + if (usage > 1000) + return 5; + if (usage > 100) + return 8; + return 10; + } + }; + + using Records = std::vector; + + Poco::Net::IPAddress selectBest() TSA_REQUIRES(mutex); + Records::iterator find(const Poco::Net::IPAddress & address) TSA_REQUIRES(mutex); + bool isUpdateNeeded(); + + void updateImpl(Poco::Timestamp now, std::vector & next_gen) TSA_REQUIRES(mutex); + void updateWeights() TSA_REQUIRES(mutex); + void updateWeightsImpl() TSA_REQUIRES(mutex); + size_t getTotalWeight() const TSA_REQUIRES(mutex); + + const String host; + const Poco::Timespan history; + const HostResolverMetrics metrics = getMetrics(); + + // for tests purpose + const ResolveFunction resolve_function; + + std::mutex mutex; + + Poco::Timestamp last_resolve_time TSA_GUARDED_BY(mutex); + Records records TSA_GUARDED_BY(mutex); + + std::uniform_int_distribution random_weight_picker TSA_GUARDED_BY(mutex); + + Poco::Logger * log = &Poco::Logger::get("ConnectionPool"); +}; + +class HostResolversPool +{ +private: + HostResolversPool() = default; + HostResolversPool(const HostResolversPool &) = delete; + HostResolversPool & operator=(const HostResolversPool &) = delete; + +public: + static HostResolversPool & instance(); + + void dropCache(); + + HostResolver::Ptr getResolver(const String & host); +private: + std::mutex mutex; + std::unordered_map host_pools TSA_GUARDED_BY(mutex); +}; + +} + diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 052c059a72d..0c9582ab4fb 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -696,6 +696,35 @@ The server successfully detected this situation and will download merged part fr \ M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \ M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas") \ + \ + M(StorageConnectionsCreated, "Number of created connections for storages") \ + M(StorageConnectionsReused, "Number of reused connections for storages") \ + M(StorageConnectionsReset, "Number of reset connections for storages") \ + M(StorageConnectionsPreserved, "Number of preserved connections for storages") \ + M(StorageConnectionsExpired, "Number of expired connections for storages") \ + M(StorageConnectionsErrors, "Number of cases when creation of a connection for storage is failed") \ + M(StorageConnectionsElapsedMicroseconds, "Total time spend on creating connections for storages") \ + \ + M(DiskConnectionsCreated, "Number of created connections for disk") \ + M(DiskConnectionsReused, "Number of reused connections for disk") \ + M(DiskConnectionsReset, "Number of reset connections for disk") \ + M(DiskConnectionsPreserved, "Number of preserved connections for disk") \ + M(DiskConnectionsExpired, "Number of expired connections for disk") \ + M(DiskConnectionsErrors, "Number of cases when creation of a connection for disk is failed") \ + M(DiskConnectionsElapsedMicroseconds, "Total time spend on creating connections for disk") \ + \ + M(HTTPConnectionsCreated, "Number of created http connections") \ + M(HTTPConnectionsReused, "Number of reused http connections") \ + M(HTTPConnectionsReset, "Number of reset http connections") \ + M(HTTPConnectionsPreserved, "Number of preserved http connections") \ + M(HTTPConnectionsExpired, "Number of expired http connections") \ + M(HTTPConnectionsErrors, "Number of cases when creation of a http connection failed") \ + M(HTTPConnectionsElapsedMicroseconds, "Total time spend on creating http connections") \ + \ + M(AddressesDiscovered, "Total count of new addresses in dns resolve results for connection pools") \ + M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for for connection pools") \ + M(AddressesFailScored, "Total count of new addresses in dns resolve results for for connection pools") \ + #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index 53e569bf6e4..11a09cb5924 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -44,11 +44,13 @@ struct ProxyConfiguration } } - std::string host; - Protocol protocol; - uint16_t port; - bool tunneling; - Protocol original_request_protocol; + std::string host = std::string{}; + Protocol protocol = Protocol::HTTP; + uint16_t port = 0; + bool tunneling = false; + Protocol original_request_protocol = Protocol::HTTP; + + bool isEmpty() const { return host.size() == 0; } }; } diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index 01a6f52185f..c9b8923929a 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -26,8 +26,6 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve() auto & proxy = proxies[index]; - LOG_DEBUG(getLogger("ProxyListConfigurationResolver"), "Use proxy: {}", proxies[index].toString()); - return ProxyConfiguration { proxy.getHost(), ProxyConfiguration::protocolFromString(proxy.getScheme()), diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index 117c8a34dbb..ef972a8e318 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -69,7 +69,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve() { auto resolved_endpoint = endpoint; resolved_endpoint.setHost(resolved_hosts[i].toString()); - session = makeHTTPSession(resolved_endpoint, timeouts); + session = makeHTTPSession(HTTPConnectionGroupType::HTTP, resolved_endpoint, timeouts); try { diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp new file mode 100644 index 00000000000..01b78958442 --- /dev/null +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -0,0 +1,558 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace +{ + +size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits::max()) +{ + const size_t buffer_size = 4096; + char buffer[buffer_size]; + + size_t total_read = 0; + + while (count > buffer_size) + { + in.read(buffer, buffer_size); + size_t read = in.gcount(); + out.write(buffer, read); + count -= read; + total_read += read; + + if (read == 0) + return total_read; + } + + in.read(buffer, count); + size_t read = in.gcount(); + out.write(buffer, read); + total_read += read; + + return total_read; +} + +class MockRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit MockRequestHandler(std::shared_ptr> slowdown_) + : slowdown(std::move(slowdown_)) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override + { + response.setStatus(Poco::Net::HTTPResponse::HTTP_OK); + auto size = request.getContentLength(); + if (size > 0) + response.setContentLength(size); // ContentLength is required for keep alive + else + response.setChunkedTransferEncoding(true); // or chunk encoding + + sleepForSeconds(*slowdown); + + stream_copy_n(request.stream(), response.send(), size); + } + + std::shared_ptr> slowdown; +}; + +class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + explicit HTTPRequestHandlerFactory(std::shared_ptr> slowdown_) + : slowdown(std::move(slowdown_)) + { + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override + { + return new MockRequestHandler(slowdown); + } + + std::shared_ptr> slowdown; +}; + +} + +using HTTPSession = Poco::Net::HTTPClientSession; +using HTTPSessionPtr = std::shared_ptr; + +class ConnectionPoolTest : public testing::Test { +protected: + ConnectionPoolTest() + { + startServer(); + } + + void SetUp() override { + timeouts = DB::ConnectionTimeouts(); + DB::HTTPConnectionPools::Limits def_limits{}; + DB::HTTPConnectionPools::instance().setLimits(def_limits, def_limits, def_limits); + + setSlowDown(0); + + DB::HTTPConnectionPools::instance().dropCache(); + DB::CurrentThread::getProfileEvents().reset(); + // Code here will be called immediately after the constructor (right + // before each test). + } + + void TearDown() override { + // Code here will be called immediately after each test (right + // before the destructor). + } + + DB::IHTTPConnectionPoolForEndpoint::Ptr getPool() + { + auto uri = Poco::URI(getServerUrl()); + return DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{}); + } + + std::string getServerUrl() const + { + return "http://" + server_data.socket->address().toString(); + } + + void startServer() + { + server_data.reset(); + server_data.params = new Poco::Net::HTTPServerParams(); + server_data.socket = std::make_unique(server_data.port); + server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive); + server_data.server = std::make_unique( + server_data.handler_factory, *server_data.socket, server_data.params); + + server_data.server->start(); + } + + Poco::Net::HTTPServer & getServer() const + { + return *server_data.server; + } + + void setSlowDown(size_t seconds) + { + *slowdown_receive = seconds; + } + + DB::ConnectionTimeouts timeouts; + std::shared_ptr> slowdown_receive = std::make_shared>(0); + + struct ServerData + { + // just some port to avoid collisions with others tests + UInt16 port = 9871; + Poco::Net::HTTPServerParams::Ptr params; + std::unique_ptr socket; + HTTPRequestHandlerFactory::Ptr handler_factory; + std::unique_ptr server; + + ServerData() = default; + ServerData(ServerData &&) = default; + ServerData & operator =(ServerData &&) = delete; + + void reset() + { + if (server) + server->stop(); + + server = nullptr; + handler_factory = nullptr; + socket = nullptr; + params = nullptr; + } + + ~ServerData() { + reset(); + } + }; + + ServerData server_data; +}; + + +void wait_until(std::function pred) +{ + while (!pred()) + sleepForMilliseconds(250); +} + +void echoRequest(String data, HTTPSession & session) +{ + { + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_PUT, "/", "HTTP/1.1"); // HTTP/1.1 is required for keep alive + request.setContentLength(data.size()); + std::ostream & ostream = session.sendRequest(request); + ostream << data; + } + + { + std::stringstream result; + Poco::Net::HTTPResponse response; + std::istream & istream = session.receiveResponse(response); + ASSERT_EQ(response.getStatus(), Poco::Net::HTTPResponse::HTTP_OK); + + stream_copy_n(istream, result); + ASSERT_EQ(data, result.str()); + } +} + +TEST_F(ConnectionPoolTest, CanConnect) +{ + auto pool = getPool(); + auto connection = pool->getConnection(timeouts); + + ASSERT_TRUE(connection->connected()); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); +} + +TEST_F(ConnectionPoolTest, CanRequest) +{ + auto pool = getPool(); + auto connection = pool->getConnection(timeouts); + + echoRequest("Hello", *connection); + + ASSERT_EQ(1, getServer().totalConnections()); + ASSERT_EQ(1, getServer().currentConnections()); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); +} + +TEST_F(ConnectionPoolTest, CanPreserve) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + } + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + } + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); + + { + auto connection = pool->getConnection(timeouts); + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + + echoRequest("Hello", *connection); + + ASSERT_EQ(1, getServer().totalConnections()); + ASSERT_EQ(1, getServer().currentConnections()); + + connection->reset(); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse10) +{ + auto pool = getPool(); + + + for (int i = 0; i < 10; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + + { + auto connection = pool->getConnection(timeouts); + connection->reset(); // reset just not to wait its expiration here + } + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse5) +{ + timeouts.withHTTPKeepAliveTimeout(1); + + auto pool = getPool(); + + std::vector connections; + connections.reserve(5); + for (int i = 0; i < 5; ++i) + { + connections.push_back(pool->getConnection(timeouts)); + } + connections.clear(); + + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 5; }); + ASSERT_EQ(5, getServer().currentConnections()); + ASSERT_EQ(5, getServer().totalConnections()); + + for (int i = 0; i < 5; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + + ASSERT_EQ(5, getServer().totalConnections()); + + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); +} + +TEST_F(ConnectionPoolTest, CanReconnectAndCreate) +{ + auto pool = getPool(); + + std::vector in_use; + + const size_t count = 2; + for (int i = 0; i < count; ++i) + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + in_use.push_back(connection); + } + + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + + ASSERT_EQ(count, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + auto connection = std::move(in_use.back()); + in_use.pop_back(); + + echoRequest("Hello", *connection); + + connection->abort(); // further usage requires reconnect, new connection + + echoRequest("Hello", *connection); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + ASSERT_EQ(count+1, getServer().totalConnections()); + + ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReconnectAndReuse) +{ + auto pool = getPool(); + + std::vector in_use; + + const size_t count = 2; + for (int i = 0; i < count; ++i) + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + in_use.push_back(std::move(connection)); + } + + auto connection = std::move(in_use.back()); + in_use.pop_back(); + in_use.clear(); // other connection will be reused + + echoRequest("Hello", *connection); + + connection->abort(); // further usage requires reconnect, reuse connection from pool + + echoRequest("Hello", *connection); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(2, getServer().totalConnections()); + + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, ReceiveTimeout) +{ + setSlowDown(2); + timeouts.withReceiveTimeout(1); + + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + ASSERT_ANY_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + + { + timeouts.withReceiveTimeout(3); + auto connection = pool->getConnection(timeouts); + ASSERT_NO_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + + { + /// timeouts have effect for reused session + timeouts.withReceiveTimeout(1); + auto connection = pool->getConnection(timeouts); + ASSERT_ANY_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} + +TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP) +{ + std::string_view message = "Hello ReadWriteBufferFromHTTP"; + auto uri = Poco::URI(getServerUrl()); + auto metrics = DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{})->getMetrics(); + Poco::Net::HTTPBasicCredentials empty_creds; + auto buf_from_http = DB::BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withOutCallback( + [&] (std::ostream & in) + { + in << message; + }) + .withDelayInit(false) + .create(empty_creds); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); + + char buf[256]; + std::fill(buf, buf + sizeof(buf), 0); + + buf_from_http->readStrict(buf, message.size()); + ASSERT_EQ(std::string_view(buf), message); + ASSERT_TRUE(buf_from_http->eof()); + + buf_from_http.reset(); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); +} + +TEST_F(ConnectionPoolTest, HardLimit) +{ + DB::HTTPConnectionPools::Limits zero_limits {0, 0, 0}; + DB::HTTPConnectionPools::instance().setLimits(zero_limits, zero_limits, zero_limits); + + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} diff --git a/src/Common/tests/gtest_resolve_pool.cpp b/src/Common/tests/gtest_resolve_pool.cpp new file mode 100644 index 00000000000..25e867fdebc --- /dev/null +++ b/src/Common/tests/gtest_resolve_pool.cpp @@ -0,0 +1,278 @@ +#include +#include +#include + +#include +#include + +class ResolvePoolMock : public DB::HostResolver +{ +public: + using ResolveFunction = DB::HostResolver::ResolveFunction; + + ResolvePoolMock(String host_, Poco::Timespan history_, ResolveFunction && func) + : DB::HostResolver(std::move(func), std::move(host_), history_) + { + } +}; + +class ResolvePoolTest : public testing::Test +{ +protected: + ResolvePoolTest() + { + DB::HostResolversPool::instance().dropCache(); + } + + void SetUp() override { + DB::CurrentThread::getProfileEvents().reset(); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + + addresses = std::set{"127.0.0.1", "127.0.0.2", "127.0.0.3"}; + // Code here will be called immediately after the constructor (right + // before each test). + } + + void TearDown() override { + // Code here will be called immediately after each test (right + // before the destructor). + } + + DB::HostResolver::Ptr make_resolver(size_t history_ms = 200) + { + auto resolve_func = [&] (const String &) + { + std::vector result; + result.reserve(addresses.size()); + for (const auto & item : addresses) + { + result.push_back(Poco::Net::IPAddress(item)); + } + return result; + }; + + + return std::make_shared("some_host", Poco::Timespan(history_ms * 1000), std::move(resolve_func)); + } + + DB::HostResolverMetrics metrics = DB::HostResolver::getMetrics(); + std::set addresses; +}; + +TEST_F(ResolvePoolTest, CanResolve) +{ + auto resolver = make_resolver(); + auto address = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*address)); + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count)); +} + +TEST_F(ResolvePoolTest, CanResolveAll) +{ + auto resolver = make_resolver(); + + std::set results; + while (results.size() != addresses.size()) + { + auto next_addr = resolver->resolve(); + results.insert(*next_addr); + } + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); +} + +size_t getSum(std::map container) +{ + size_t sum = 0; + for (auto & [_, val] : container) + { + sum += val; + } + return sum; +} + +size_t getMin(std::map container) +{ + if (container.empty()) + return 0; + + size_t min_val = container.begin()->second; + for (auto & [_, val] : container) + { + min_val = std::min(min_val, val); + } + return min_val; +} + +double getMean(std::map container) +{ + return 1.0 * getSum(container) / container.size(); +} + +double getMaxDiff(std::map container, double ref_val) +{ + double diff = 0.0; + for (auto & [_, val] : container) + { + diff = std::max(std::fabs(val - ref_val), diff); + } + + return diff; +} + +TEST_F(ResolvePoolTest, CanResolveEvenly) +{ + auto resolver = make_resolver(); + + std::map results; + + for (size_t i = 0; i < 50000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + auto mean = getMean(results); + auto diff = getMaxDiff(results, mean); + + ASSERT_GT(0.3 * mean, diff); +} + +TEST_F(ResolvePoolTest, CanMerge) +{ + auto resolver = make_resolver(100000); + auto address = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*address)); + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + + auto old_addresses = addresses; + addresses = std::set{"127.0.0.4", "127.0.0.5"}; + + + resolver->update(); + ASSERT_EQ(addresses.size() + old_addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(addresses.size() + old_addresses.size(), CurrentMetrics::get(metrics.active_count)); + + std::set results; + while (results.size() != addresses.size() + old_addresses.size()) + { + auto next_addr = resolver->resolve(); + results.insert(*next_addr); + } +} + +TEST_F(ResolvePoolTest, CanGainEven) +{ + auto resolver = make_resolver(); + auto address = resolver->resolve(); + + std::map results; + for (size_t i = 0; i < 40000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + ASSERT_GT(getMin(results), 10000); + + addresses.insert("127.0.0.4"); + addresses.insert("127.0.0.5"); + + resolver->update(); + + /// return mostly new addresses + for (size_t i = 0; i < 3000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + ASSERT_EQ(results.size(), 5); + + ASSERT_GT(getMin(results), 1000); +} + +TEST_F(ResolvePoolTest, CanFail) +{ + auto resolver = make_resolver(10000); + + auto failed_addr = resolver->resolve(); + failed_addr.setFail(); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.failed]); + ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + + for (size_t i = 0; i < 1000; ++i) + { + auto next_addr = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*next_addr)); + ASSERT_NE(*next_addr, *failed_addr); + } +} + +TEST_F(ResolvePoolTest, CanFailAndHeal) +{ + auto resolver = make_resolver(); + + auto failed_addr = resolver->resolve(); + failed_addr.setFail(); + + while (true) + { + auto next_addr = resolver->resolve(); + if (*failed_addr == *next_addr) + break; + } +} + + +TEST_F(ResolvePoolTest, CanExpire) +{ + auto resolver = make_resolver(); + + auto expired_addr = resolver->resolve(); + ASSERT_TRUE(addresses.contains(*expired_addr)); + + addresses.erase(*expired_addr); + sleepForSeconds(1); + + for (size_t i = 0; i < 1000; ++i) + { + auto next_addr = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*next_addr)); + ASSERT_NE(*next_addr, *expired_addr); + } + + ASSERT_EQ(addresses.size() + 1, DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]); +} diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index fc478ae4f41..c201bab6063 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -128,8 +128,17 @@ namespace DB M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \ M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \ M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \ + M(UInt64, disk_connections_soft_limit, 1000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_store_limit, 12000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \ + M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \ + M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \ + M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \ + M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ + M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ + M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ - /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp +/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fceee63d4bb..d70a6cf51c5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -110,7 +110,7 @@ class IColumn; M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ - M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. This only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ + M(UInt64, s3_connect_timeout_ms, 1000, "Connection timeout for host from s3 disks.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 2f1da7935e6..e680c02671a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"24.3", {{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, + {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, + {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index bf42b7931ed..dae8ec06d30 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -88,20 +88,18 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) QueryPipeline HTTPDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); - Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + Poco::URI uri(configuration.url); + + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadUpdatedAll() @@ -109,19 +107,16 @@ QueryPipeline HTTPDictionarySource::loadUpdatedAll() Poco::URI uri(configuration.url); getUpdateFieldAndDate(uri); LOG_TRACE(log, "loadUpdatedAll {}", uri.toString()); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) @@ -139,19 +134,18 @@ QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) }; Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - out_stream_callback, - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withOutCallback(std::move(out_stream_callback)) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) @@ -169,19 +163,18 @@ QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const }; Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - out_stream_callback, - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withOutCallback(std::move(out_stream_callback)) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } bool HTTPDictionarySource::isModified() const diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 70fe889a8ea..1ebfc4a29b0 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -203,7 +203,7 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) } -QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const +QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const { bridge_helper->startBridgeSync(); @@ -214,10 +214,15 @@ QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const B os << "query=" << escapeForFileName(query); }; - auto read_buf = std::make_unique( - url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts, credentials); - auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, max_block_size); - format->addBuffer(std::move(read_buf)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(timeouts) + .withOutCallback(std::move(write_body_callback)) + .create(credentials); + + auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *buf, required_sample_block, max_block_size); + format->addBuffer(std::move(buf)); return QueryPipeline(std::move(format)); } diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 6011563c522..64d22807254 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -74,7 +74,7 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - QueryPipeline loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; + QueryPipeline loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const; LoggerPtr log; diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index cc872392738..7509aa81d75 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -1,8 +1,6 @@ #include "ReadBufferFromWebServer.h" #include -#include -#include #include #include #include @@ -45,12 +43,6 @@ std::unique_ptr ReadBufferFromWebServer::initialize() { if (read_until_position < offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - - LOG_DEBUG(log, "Reading with range: {}-{}", offset, read_until_position); - } - else - { - LOG_DEBUG(log, "Reading from offset: {}", offset); } const auto & settings = context->getSettingsRef(); @@ -60,19 +52,14 @@ std::unique_ptr ReadBufferFromWebServer::initialize() connection_timeouts.withConnectionTimeout(std::max(settings.http_connection_timeout, Poco::Timespan(20, 0))); connection_timeouts.withReceiveTimeout(std::max(settings.http_receive_timeout, Poco::Timespan(20, 0))); - auto res = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - connection_timeouts, - credentials, - 0, - buf_size, - read_settings, - HTTPHeaderEntries{}, - &context->getRemoteHostFilter(), - /* delay_initialization */true, - use_external_buffer); + auto res = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::DISK) + .withSettings(read_settings) + .withTimeouts(connection_timeouts) + .withBufSize(buf_size) + .withHostFilter(&context->getRemoteHostFilter()) + .withExternalBuf(use_external_buffer) + .create(credentials); if (read_until_position) res->setReadUntilPosition(read_until_position); @@ -101,44 +88,44 @@ bool ReadBufferFromWebServer::nextImpl() throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); } - if (impl) - { - if (!use_external_buffer) - { - /** - * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read, because - * this branch means we read sequentially. - */ - impl->position() = position(); - assert(!impl->hasPendingData()); - } - } - else + if (!impl) { impl = initialize(); + + if (!use_external_buffer) + { + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + } } if (use_external_buffer) { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not, because this branch - * means we are prefetching data, each nextImpl() call we can fill - * a different buffer. - */ impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); + } + else + { + impl->position() = position(); } + chassert(available() == 0); + + chassert(pos >= working_buffer.begin()); + chassert(pos <= working_buffer.end()); + + chassert(working_buffer.begin() != nullptr); + chassert(impl->buffer().begin() != nullptr); + chassert(working_buffer.begin() == impl->buffer().begin()); + + chassert(impl->available() == 0); + auto result = impl->next(); + + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + + chassert(working_buffer.begin() == impl->buffer().begin()); + if (result) - { - BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); offset += working_buffer.size(); - } return result; } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4fd4b17aabe..7cc29bf1da2 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -67,10 +67,6 @@ std::unique_ptr getClient( client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms = config.getUInt( - config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); - client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); - client_configuration.wait_on_pool_size_limit = false; client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 48de0bf4168..0bad668a404 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -44,34 +44,33 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lockgetSettingsRef(), - getContext()->getServerSettings().keep_alive_timeout), - credentials, - /* max_redirects= */ 0, - /* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE, - getContext()->getReadSettings()); + auto timeouts = ConnectionTimeouts::getHTTPTimeouts( + getContext()->getSettingsRef(), + getContext()->getServerSettings().keep_alive_timeout); + + auto metadata_buf = BuilderRWBufferFromHTTP(Poco::URI(fs::path(full_url) / ".index")) + .withConnectionGroup(HTTPConnectionGroupType::DISK) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .create(credentials); String file_name; - while (!metadata_buf.eof()) + while (!metadata_buf->eof()) { - readText(file_name, metadata_buf); - assertChar('\t', metadata_buf); + readText(file_name, *metadata_buf); + assertChar('\t', *metadata_buf); bool is_directory; - readBoolText(is_directory, metadata_buf); + readBoolText(is_directory, *metadata_buf); size_t size = 0; if (!is_directory) { - assertChar('\t', metadata_buf); - readIntText(size, metadata_buf); + assertChar('\t', *metadata_buf); + readIntText(size, *metadata_buf); } - assertChar('\n', metadata_buf); + assertChar('\n', *metadata_buf); FileDataPtr file_data = is_directory ? FileData::createDirectoryInfo(false) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index a052f20d6fa..6679646fef4 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -122,7 +122,7 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { std::string_view sqid = col_non_const->getDataAt(i).toView(); - std::vector integers = sqids.decode(sqid); + std::vector integers = sqids.decode(String(sqid)); res_nested_data.insert(integers.begin(), integers.end()); res_offsets_data.push_back(integers.size()); } diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index f2db3169400..c4b636103fe 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -141,4 +141,19 @@ ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method .withReceiveTimeout(saturate(recv, receive_timeout)); } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) +{ + session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); +} + +ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session) +{ + return ConnectionTimeouts() + .withConnectionTimeout(session.getConnectionTimeout()) + .withSendTimeout(session.getSendTimeout()) + .withReceiveTimeout(session.getReceiveTimeout()) + .withHTTPKeepAliveTimeout(session.getKeepAliveTimeout()); +} + } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 7fe97b5ec36..2b2ab0e7ab8 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -4,6 +4,7 @@ #include #include +#include #include namespace DB @@ -111,4 +112,7 @@ inline ConnectionTimeouts & ConnectionTimeouts::withConnectionTimeout(Poco::Time return *this; } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); +ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session); + } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index c4468a1b896..09f7724d613 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -2,13 +2,7 @@ #include #include -#include -#include #include -#include -#include -#include -#include #include "config.h" @@ -25,338 +19,18 @@ #include -#include -#include +#include #include #include -namespace ProfileEvents -{ - extern const Event CreatedHTTPConnections; -} - namespace DB { + namespace ErrorCodes { extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER; extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; - extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME; - extern const int UNSUPPORTED_URI_SCHEME; - extern const int LOGICAL_ERROR; -} - - -namespace -{ - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration) - { - Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; - - poco_proxy_config.host = proxy_configuration.host; - poco_proxy_config.port = proxy_configuration.port; - poco_proxy_config.protocol = ProxyConfiguration::protocolToString(proxy_configuration.protocol); - poco_proxy_config.tunnel = proxy_configuration.tunneling; - poco_proxy_config.originalRequestProtocol = ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); - - return poco_proxy_config; - } - - template - requires std::derived_from - class HTTPSessionAdapter : public Session - { - static_assert(std::has_virtual_destructor_v, "The base class must have a virtual destructor"); - - public: - HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{getLogger("HTTPSessionAdapter")} { } - ~HTTPSessionAdapter() override = default; - - protected: - void reconnect() override - { - // First of all will try to establish connection with last used addr. - if (!Session::getResolvedHost().empty()) - { - try - { - Session::reconnect(); - return; - } - catch (...) - { - Session::close(); - LOG_TRACE( - log, - "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", - Session::getResolvedHost(), - Session::getHost(), - Session::getPort()); - } - } - - const auto endpoinds = DNSResolver::instance().resolveHostAll(Session::getHost()); - - for (auto it = endpoinds.begin();;) - { - try - { - Session::setResolvedHost(it->toString()); - Session::reconnect(); - - LOG_TRACE( - log, - "Created HTTP(S) session with {}:{} ({}:{})", - Session::getHost(), - Session::getPort(), - it->toString(), - Session::getPort()); - - break; - } - catch (...) - { - Session::close(); - if (++it == endpoinds.end()) - { - Session::setResolvedHost(""); - throw; - } - LOG_TRACE( - log, - "Failed to create connection with {}:{}, Will try another resolved address. {}", - Session::getResolvedHost(), - Session::getPort(), - getCurrentExceptionMessage(false)); - } - } - } - LoggerPtr log; - }; - - bool isHTTPS(const Poco::URI & uri) - { - if (uri.getScheme() == "https") - return true; - else if (uri.getScheme() == "http") - return false; - else - throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); - } - - HTTPSessionPtr makeHTTPSessionImpl( - const std::string & host, - UInt16 port, - bool https, - bool keep_alive, - DB::ProxyConfiguration proxy_configuration = {}) - { - HTTPSessionPtr session; - - if (!proxy_configuration.host.empty()) - { - bool is_proxy_http_and_is_tunneling_off = DB::ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol - && !proxy_configuration.tunneling; - - // If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request. - // The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server. - // There is a weak link in the security, but that's what the user opted for. - if (https && is_proxy_http_and_is_tunneling_off) - { - https = false; - } - } - - if (https) - { -#if USE_SSL - session = std::make_shared>(host, port); -#else - throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without HTTPS support"); -#endif - } - else - { - session = std::make_shared>(host, port); - } - - ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - - /// doesn't work properly without patch - session->setKeepAlive(keep_alive); - - if (!proxy_configuration.host.empty()) - { - session->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); - } - - return session; - } - - class SingleEndpointHTTPSessionPool : public PoolBase - { - private: - const std::string host; - const UInt16 port; - const bool https; - ProxyConfiguration proxy_config; - - using Base = PoolBase; - - ObjectPtr allocObject() override - { - /// Pool is global, we shouldn't attribute this memory to query/user. - MemoryTrackerSwitcher switcher{&total_memory_tracker}; - - auto session = makeHTTPSessionImpl(host, port, https, true, proxy_config); - return session; - } - - public: - SingleEndpointHTTPSessionPool( - const std::string & host_, - UInt16 port_, - bool https_, - ProxyConfiguration proxy_config_, - size_t max_pool_size_, - bool wait_on_pool_size_limit) - : Base( - static_cast(max_pool_size_), - getLogger("HTTPSessionPool"), - wait_on_pool_size_limit ? BehaviourOnLimit::Wait : BehaviourOnLimit::AllocateNewBypassingPool) - , host(host_) - , port(port_) - , https(https_) - , proxy_config(proxy_config_) - { - } - }; - - class HTTPSessionPool : private boost::noncopyable - { - public: - struct Key - { - String target_host; - UInt16 target_port; - bool is_target_https; - ProxyConfiguration proxy_config; - bool wait_on_pool_size_limit; - - bool operator ==(const Key & rhs) const - { - return std::tie( - target_host, - target_port, - is_target_https, - proxy_config.host, - proxy_config.port, - proxy_config.protocol, - proxy_config.tunneling, - proxy_config.original_request_protocol, - wait_on_pool_size_limit) - == std::tie( - rhs.target_host, - rhs.target_port, - rhs.is_target_https, - rhs.proxy_config.host, - rhs.proxy_config.port, - rhs.proxy_config.protocol, - rhs.proxy_config.tunneling, - rhs.proxy_config.original_request_protocol, - rhs.wait_on_pool_size_limit); - } - }; - - private: - using PoolPtr = std::shared_ptr; - using Entry = SingleEndpointHTTPSessionPool::Entry; - - struct Hasher - { - size_t operator()(const Key & k) const - { - SipHash s; - s.update(k.target_host); - s.update(k.target_port); - s.update(k.is_target_https); - s.update(k.proxy_config.host); - s.update(k.proxy_config.port); - s.update(k.proxy_config.protocol); - s.update(k.proxy_config.tunneling); - s.update(k.proxy_config.original_request_protocol); - s.update(k.wait_on_pool_size_limit); - return s.get64(); - } - }; - - std::mutex mutex; - std::unordered_map endpoints_pool; - - protected: - HTTPSessionPool() = default; - - public: - static auto & instance() - { - static HTTPSessionPool instance; - return instance; - } - - Entry getSession( - const Poco::URI & uri, - const ProxyConfiguration & proxy_config, - const ConnectionTimeouts & timeouts, - size_t max_connections_per_endpoint, - bool wait_on_pool_size_limit) - { - std::unique_lock lock(mutex); - const std::string & host = uri.getHost(); - UInt16 port = uri.getPort(); - bool https = isHTTPS(uri); - - HTTPSessionPool::Key key{host, port, https, proxy_config, wait_on_pool_size_limit}; - auto pool_ptr = endpoints_pool.find(key); - if (pool_ptr == endpoints_pool.end()) - std::tie(pool_ptr, std::ignore) = endpoints_pool.emplace( - key, - std::make_shared( - host, - port, - https, - proxy_config, - max_connections_per_endpoint, - wait_on_pool_size_limit)); - - /// Some routines held session objects until the end of its lifetime. Also this routines may create another sessions in this time frame. - /// If some other session holds `lock` because it waits on another lock inside `pool_ptr->second->get` it isn't possible to create any - /// new session and thus finish routine, return session to the pool and unlock the thread waiting inside `pool_ptr->second->get`. - /// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`. - lock.unlock(); - - auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); - auto session = pool_ptr->second->get(retry_timeout); - - const auto & session_data = session->sessionData(); - if (session_data.empty() || !Poco::AnyCast(&session_data)) - { - /// Reset session if it is not reusable. See comment for HTTPSessionReuseTag. - session->reset(); - } - session->attachSessionData({}); - - setTimeouts(*session, timeouts); - - return session; - } - }; -} - -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) -{ - session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); } void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) @@ -370,28 +44,13 @@ void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_ } HTTPSessionPtr makeHTTPSession( + HTTPConnectionGroupType group, const Poco::URI & uri, const ConnectionTimeouts & timeouts, - ProxyConfiguration proxy_configuration -) + ProxyConfiguration proxy_configuration) { - const std::string & host = uri.getHost(); - UInt16 port = uri.getPort(); - bool https = isHTTPS(uri); - - auto session = makeHTTPSessionImpl(host, port, https, false, proxy_configuration); - setTimeouts(*session, timeouts); - return session; -} - -PooledHTTPSessionPtr makePooledHTTPSession( - const Poco::URI & uri, - const ConnectionTimeouts & timeouts, - size_t per_endpoint_pool_size, - bool wait_on_pool_size_limit, - ProxyConfiguration proxy_config) -{ - return HTTPSessionPool::instance().getSession(uri, proxy_config, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); + auto connection_pool = HTTPConnectionPools::instance().getPool(group, uri, proxy_configuration); + return connection_pool->getConnection(timeouts); } bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; } @@ -400,11 +59,11 @@ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, const bool allow_redirects) { auto & istr = session.receiveResponse(response); - assertResponseIsOk(request, response, istr, allow_redirects); + assertResponseIsOk(request.getURI(), response, istr, allow_redirects); return &istr; } -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects) +void assertResponseIsOk(const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects) { auto status = response.getStatus(); @@ -422,7 +81,7 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR body.exceptions(std::ios::failbit); body << istr.rdbuf(); - throw HTTPException(code, request.getURI(), status, response.getReason(), body.str()); + throw HTTPException(code, uri, status, response.getReason(), body.str()); } } @@ -440,24 +99,4 @@ Exception HTTPException::makeExceptionMessage( uri, static_cast(http_status), reason, body); } -void markSessionForReuse(Poco::Net::HTTPSession & session) -{ - const auto & session_data = session.sessionData(); - if (!session_data.empty() && !Poco::AnyCast(&session_data)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Data of an unexpected type ({}) is attached to the session", session_data.type().name()); - - session.attachSessionData(HTTPSessionReuseTag{}); -} - -void markSessionForReuse(HTTPSessionPtr session) -{ - markSessionForReuse(*session); -} - -void markSessionForReuse(PooledHTTPSessionPtr session) -{ - markSessionForReuse(static_cast(*session)); -} - } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index c9968fc6915..e27269e2559 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -7,9 +7,9 @@ #include #include #include -#include -#include #include +#include +#include #include @@ -36,7 +36,7 @@ public: HTTPException * clone() const override { return new HTTPException(*this); } void rethrow() const override { throw *this; } - int getHTTPStatus() const { return http_status; } + Poco::Net::HTTPResponse::HTTPStatus getHTTPStatus() const { return http_status; } private: Poco::Net::HTTPResponse::HTTPStatus http_status{}; @@ -52,55 +52,18 @@ private: const char * className() const noexcept override { return "DB::HTTPException"; } }; -using PooledHTTPSessionPtr = PoolBase::Entry; // SingleEndpointHTTPSessionPool::Entry using HTTPSessionPtr = std::shared_ptr; -/// If a session have this tag attached, it will be reused without calling `reset()` on it. -/// All pooled sessions don't have this tag attached after being taken from a pool. -/// If the request and the response were fully written/read, the client code should add this tag -/// explicitly by calling `markSessionForReuse()`. -/// -/// Note that HTTP response may contain extra bytes after the last byte of the payload. Specifically, -/// when chunked encoding is used, there's an empty chunk at the end. Those extra bytes must also be -/// read before the session can be reused. So we usually put an `istr->ignore(INT64_MAX)` call -/// before `markSessionForReuse()`. -struct HTTPSessionReuseTag -{ -}; - -void markSessionForReuse(Poco::Net::HTTPSession & session); -void markSessionForReuse(HTTPSessionPtr session); -void markSessionForReuse(PooledHTTPSessionPtr session); - - void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession( + HTTPConnectionGroupType group, const Poco::URI & uri, const ConnectionTimeouts & timeouts, ProxyConfiguration proxy_config = {} ); -/// As previous method creates session, but takes it from pool, without and with proxy uri. -/// -/// The max_connections_per_endpoint parameter makes it look like the pool size can be different for -/// different requests (whatever that means), but actually we just assign the endpoint's connection -/// pool size when we see the endpoint for the first time, then we never change it. -/// We should probably change how this configuration works, and how this pooling works in general: -/// * Make the per_endpoint_pool_size be a global server setting instead of per-disk or per-query. -/// * Have boolean per-disk/per-query settings for enabling/disabling pooling. -/// * Add a limit on the number of endpoints and the total number of sessions across all endpoints. -/// * Enable pooling by default everywhere. In particular StorageURL and StorageS3. -/// (Enabling it for StorageURL is scary without the previous item - the user may query lots of -/// different endpoints. So currently pooling is mainly used for S3.) -PooledHTTPSessionPtr makePooledHTTPSession( - const Poco::URI & uri, - const ConnectionTimeouts & timeouts, - size_t per_endpoint_pool_size, - bool wait_on_pool_size_limit = true, - ProxyConfiguration proxy_config = {}); - bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status); /** Used to receive response (response headers and possibly body) @@ -112,7 +75,6 @@ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects); void assertResponseIsOk( - const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); + const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); } diff --git a/src/IO/LimitReadBuffer.cpp b/src/IO/LimitReadBuffer.cpp index e14112f8d19..84c7ac86227 100644 --- a/src/IO/LimitReadBuffer.cpp +++ b/src/IO/LimitReadBuffer.cpp @@ -1,5 +1,4 @@ #include - #include @@ -15,7 +14,7 @@ namespace ErrorCodes bool LimitReadBuffer::nextImpl() { - assert(position() >= in->position()); + chassert(position() >= in->position()); /// Let underlying buffer calculate read bytes in `next()` call. in->position() = position(); @@ -39,20 +38,18 @@ bool LimitReadBuffer::nextImpl() if (exact_limit && bytes != *exact_limit) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, *exact_limit); /// Clearing the buffer with existing data. - set(in->position(), 0); + BufferBase::set(in->position(), 0, 0); + return false; } - working_buffer = in->buffer(); - - if (limit - bytes < working_buffer.size()) - working_buffer.resize(limit - bytes); + BufferBase::set(in->position(), std::min(in->available(), limit - bytes), 0); return true; } -LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : ReadBuffer(in_ ? in_->position() : nullptr, 0) , in(in_) @@ -62,24 +59,20 @@ LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, boo , exact_limit(exact_limit_) , exception_message(std::move(exception_message_)) { - assert(in); + chassert(in); - size_t remaining_bytes_in_buffer = in->buffer().end() - in->position(); - if (remaining_bytes_in_buffer > limit) - remaining_bytes_in_buffer = limit; - - working_buffer = Buffer(in->position(), in->position() + remaining_bytes_in_buffer); + BufferBase::set(in->position(), std::min(in->available(), limit), 0); } -LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_) { } -LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_) { diff --git a/src/IO/LimitReadBuffer.h b/src/IO/LimitReadBuffer.h index 15885c1d850..b869f2935fb 100644 --- a/src/IO/LimitReadBuffer.h +++ b/src/IO/LimitReadBuffer.h @@ -13,22 +13,24 @@ namespace DB class LimitReadBuffer : public ReadBuffer { public: - LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, + LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_ = {}); - LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, + LimitReadBuffer(std::unique_ptr in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_ = {}); ~LimitReadBuffer() override; private: ReadBuffer * in; - bool owns_in; + const bool owns_in; - UInt64 limit; - bool throw_exception; - std::optional exact_limit; - std::string exception_message; + const size_t limit; + const bool throw_exception; + const std::optional exact_limit; + const std::string exception_message; - LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_); + LoggerPtr log; + + LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_); bool nextImpl() override; }; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 9b1c132cc01..f27828f71b2 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -92,7 +92,7 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(getFD(), getFileName()); } -size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) const { if (offset >= mapped.getLength()) return 0; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 2a039e04971..f774538374a 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -40,7 +40,7 @@ public: size_t getFileSize() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) const override; bool supportsReadAt() override { return true; } }; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 00325734354..056e25a5fbe 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -63,21 +63,23 @@ public: */ bool next() { - assert(!hasPendingData()); - assert(position() <= working_buffer.end()); + chassert(!hasPendingData()); + chassert(position() <= working_buffer.end()); bytes += offset(); bool res = nextImpl(); if (!res) + { working_buffer = Buffer(pos, pos); + } else { - pos = working_buffer.begin() + nextimpl_working_buffer_offset; - assert(position() != working_buffer.end()); + pos = working_buffer.begin() + std::min(nextimpl_working_buffer_offset, working_buffer.size()); + chassert(position() < working_buffer.end()); } nextimpl_working_buffer_offset = 0; - assert(position() <= working_buffer.end()); + chassert(position() <= working_buffer.end()); return res; } diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 3211f8eeb35..57442a15853 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -49,7 +49,7 @@ std::string ReadBufferFromFileDescriptor::getFileName() const } -size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) +size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const { chassert(min_bytes <= max_bytes); @@ -265,7 +265,7 @@ bool ReadBufferFromFileDescriptor::checkIfActuallySeekable() return res == 0 && S_ISREG(stat.st_mode); } -size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) const { chassert(use_pread); return readImpl(to, n, n, offset); diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 4762998c67b..db256ef91c7 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -34,7 +34,7 @@ protected: /// Doesn't seek (`offset` must match fd's position if !use_pread). /// Stops after min_bytes or eof. Returns 0 if eof. /// Thread safe. - size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset); + size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const; public: explicit ReadBufferFromFileDescriptor( @@ -73,7 +73,7 @@ public: bool checkIfActuallySeekable() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) const override; bool supportsReadAt() override { return use_pread; } }; diff --git a/src/IO/ReadBufferFromIStream.cpp b/src/IO/ReadBufferFromIStream.cpp index 52546f1703d..bc90ec7ed15 100644 --- a/src/IO/ReadBufferFromIStream.cpp +++ b/src/IO/ReadBufferFromIStream.cpp @@ -5,52 +5,44 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_READ_FROM_ISTREAM; -} - bool ReadBufferFromIStream::nextImpl() { if (eof) return false; + chassert(internal_buffer.begin() != nullptr); + chassert(!internal_buffer.empty()); + size_t bytes_read = 0; char * read_to = internal_buffer.begin(); /// It is necessary to read in a loop, since socket usually returns only data available at the moment. while (bytes_read < internal_buffer.size()) { - try + const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read); + if (bytes_read_last_time <= 0) { - const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read); - if (bytes_read_last_time <= 0) - { - eof = true; - break; - } + eof = true; + break; + } - bytes_read += bytes_read_last_time; - read_to += bytes_read_last_time; - } - catch (...) - { - throw Exception( - ErrorCodes::CANNOT_READ_FROM_ISTREAM, - "Cannot read from istream at offset {}: {}", - count(), - getCurrentExceptionMessage(/*with_stacktrace=*/true)); - } + bytes_read += bytes_read_last_time; + read_to += bytes_read_last_time; } if (bytes_read) + { + working_buffer = internal_buffer; working_buffer.resize(bytes_read); + } return bytes_read; } ReadBufferFromIStream::ReadBufferFromIStream(std::istream & istr_, size_t size) - : BufferWithOwnMemory(size), istr(istr_), stream_buf(dynamic_cast(*istr.rdbuf())) + : BufferWithOwnMemory(size) + , istr(istr_) + , stream_buf(dynamic_cast(*istr.rdbuf())) { } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 4529771e7b2..491ff253066 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -34,61 +34,6 @@ namespace ProfileEvents extern const Event RemoteReadThrottlerSleepMicroseconds; } -namespace -{ -DB::PooledHTTPSessionPtr getSession(Aws::S3::Model::GetObjectResult & read_result) -{ - if (auto * session_aware_stream = dynamic_cast *>(&read_result.GetBody())) - return static_cast(session_aware_stream->getSession()); - - if (dynamic_cast *>(&read_result.GetBody())) - return {}; - - /// accept result from S# mock in gtest_writebuffer_s3.cpp - if (dynamic_cast(&read_result.GetBody())) - return {}; - - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session of unexpected type encountered"); -} - -void resetSession(Aws::S3::Model::GetObjectResult & read_result) -{ - if (auto session = getSession(read_result); !session.isNull()) - { - auto & http_session = static_cast(*session); - http_session.reset(); - } -} - -void resetSessionIfNeeded(bool read_all_range_successfully, std::optional & read_result) -{ - if (!read_result) - return; - - if (!read_all_range_successfully) - { - /// When we abandon a session with an ongoing GetObject request and there is another one trying to delete the same object this delete - /// operation will hang until GetObject's session idle timeouts. So we have to call `reset()` on GetObject's session session immediately. - resetSession(*read_result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - else if (auto session = getSession(*read_result); !session.isNull()) - { - if (!session->getProxyHost().empty()) - { - /// Reset proxified sessions because proxy can change for every request. See ProxyConfigurationResolver. - resetSession(*read_result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - else - { - DB::markSessionForReuse(session); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); - } - } -} -} - namespace DB { namespace ErrorCodes @@ -228,7 +173,7 @@ bool ReadBufferFromS3::nextImpl() } -size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const { size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; @@ -240,29 +185,6 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); std::optional result; - /// Connection is reusable if we've read the full response. - bool session_is_reusable = false; - SCOPE_EXIT( - { - if (!result.has_value()) - return; - if (session_is_reusable) - { - auto session = getSession(*result); - if (!session.isNull()) - { - DB::markSessionForReuse(session); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); - } - else - session_is_reusable = false; - } - if (!session_is_reusable) - { - resetSession(*result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - }); try { @@ -276,9 +198,8 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - /// Read remaining bytes after the end of the payload, see HTTPSessionReuseTag. + /// Read remaining bytes after the end of the payload istr.ignore(INT64_MAX); - session_is_reusable = true; } catch (Poco::Exception & e) { @@ -451,21 +372,8 @@ bool ReadBufferFromS3::atEndOfRequestedRangeGuess() return false; } -ReadBufferFromS3::~ReadBufferFromS3() -{ - try - { - resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); - } - catch (...) - { - tryLogCurrentException(log); - } -} - std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) { - resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); read_all_range_successfully = false; /** @@ -534,10 +442,6 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si } } -bool ReadBufferFromS3::readAllRangeSuccessfully() const -{ - return read_until_position ? offset == read_until_position : read_all_range_successfully; -} } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index f28c23a71d7..003c88df7d2 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -55,7 +55,7 @@ public: bool restricted_seek_ = false, std::optional file_size = std::nullopt); - ~ReadBufferFromS3() override; + ~ReadBufferFromS3() override = default; bool nextImpl() override; @@ -74,7 +74,7 @@ public: String getFileName() const override { return bucket + "/" + key; } - size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const override; bool supportsReadAt() override { return true; } @@ -90,8 +90,6 @@ private: Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const; - bool readAllRangeSuccessfully() const; - ReadSettings read_settings; bool use_external_buffer; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 31ea45d92a9..38904df4403 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -122,7 +122,7 @@ struct ReadSettings // Resource to be used during reading ResourceLink resource_link; - size_t http_max_tries = 1; + size_t http_max_tries = 10; size_t http_retry_initial_backoff_ms = 100; size_t http_retry_max_backoff_ms = 1600; bool http_skip_not_found_url_for_globs = true; diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 15c2a0a021b..bcbec97537a 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -1,13 +1,65 @@ #include "ReadWriteBufferFromHTTP.h" #include +#include +#include + namespace ProfileEvents { extern const Event ReadBufferSeekCancelConnection; - extern const Event ReadWriteBufferFromHTTPPreservedSessions; } + +namespace +{ + +bool isRetriableError(const Poco::Net::HTTPResponse::HTTPStatus http_status) noexcept +{ + static constexpr std::array non_retriable_errors{ + Poco::Net::HTTPResponse::HTTPStatus::HTTP_BAD_REQUEST, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_IMPLEMENTED, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_METHOD_NOT_ALLOWED}; + + return std::all_of( + non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); +} + +Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) +{ + chassert(DB::isRedirect(response.getStatus())); + + auto location = response.get("Location"); + auto location_uri = Poco::URI(location); + if (!location_uri.isRelative()) + return location_uri; + /// Location header contains relative path. So we need to concatenate it + /// with path from the original URI and normalize it. + auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location); + location_uri = prev_uri; + location_uri.setPath(path); + return location_uri; +} + +class ReadBufferFromSessionResponse : public DB::ReadBufferFromIStream +{ +private: + DB::HTTPSessionPtr session; + +public: + ReadBufferFromSessionResponse(DB::HTTPSessionPtr && session_, std::istream & rstr, size_t size) + : ReadBufferFromIStream(rstr, size) + , session(std::move(session_)) + { + } +}; + +} + + namespace DB { @@ -21,94 +73,29 @@ namespace ErrorCodes extern const int UNKNOWN_FILE_SIZE; } -template -UpdatableSession::UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr session_factory_) - : max_redirects{max_redirects_} - , initial_uri(uri) - , session_factory(std::move(session_factory_)) +std::unique_ptr ReadWriteBufferFromHTTP::CallResult::transformToReadBuffer(size_t buf_size) && { - session = session_factory->buildNewSession(uri); + chassert(session); + return std::make_unique(std::move(session), *response_stream, buf_size); } -template -typename UpdatableSession::SessionPtr UpdatableSession::getSession() { return session; } - -template -void UpdatableSession::updateSession(const Poco::URI & uri) -{ - ++redirects; - if (redirects <= max_redirects) - session = session_factory->buildNewSession(uri); - else - throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, - "Too many redirects while trying to access {}." - " You can {} redirects by changing the setting 'max_http_get_redirects'." - " Example: `SET max_http_get_redirects = 10`." - " Redirects are restricted to prevent possible attack when a malicious server redirects to an internal resource, bypassing the authentication or firewall.", - initial_uri.toString(), max_redirects ? "increase the allowed maximum number of" : "allow"); -} - -template -typename UpdatableSession::SessionPtr UpdatableSession::createDetachedSession(const Poco::URI & uri) -{ - return session_factory->buildNewSession(uri); -} - -template -std::shared_ptr> UpdatableSession::clone(const Poco::URI & uri) -{ - return std::make_shared>(uri, max_redirects, session_factory); -} - - -namespace detail -{ - -static bool isRetriableError(const Poco::Net::HTTPResponse::HTTPStatus http_status) noexcept -{ - static constexpr std::array non_retriable_errors{ - Poco::Net::HTTPResponse::HTTPStatus::HTTP_BAD_REQUEST, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_IMPLEMENTED, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_METHOD_NOT_ALLOWED}; - - return std::all_of( - non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); -} - -static Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) -{ - auto location = response.get("Location"); - auto location_uri = Poco::URI(location); - if (!location_uri.isRelative()) - return location_uri; - /// Location header contains relative path. So we need to concatenate it - /// with path from the original URI and normalize it. - auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location); - location_uri = prev_uri; - location_uri.setPath(path); - return location_uri; -} - -template -bool ReadWriteBufferFromHTTPBase::withPartialContent(const HTTPRange & range) const +bool ReadWriteBufferFromHTTP::withPartialContent() const { /** * Add range header if we have some passed range * or if we want to retry GET request on purpose. */ - return range.begin || range.end || retry_with_range_header; + return read_range.begin || read_range.end || getOffset() > 0; } -template -size_t ReadWriteBufferFromHTTPBase::getOffset() const { return read_range.begin.value_or(0) + offset_from_begin_pos; } - -template -void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const +size_t ReadWriteBufferFromHTTP::getOffset() const { - request.setHost(uri_.getHost()); // use original, not resolved host name in header + return read_range.begin.value_or(0) + offset_from_begin_pos; +} + +void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, std::optional range) const +{ + request.setHost(initial_uri.getHost()); // use original, not resolved host name in header if (out_stream_callback) request.setChunkedTransferEncoding(true); @@ -125,7 +112,6 @@ void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net: range_header_value = fmt::format("bytes={}-{}", *range->begin, *range->end); else range_header_value = fmt::format("bytes={}-", *range->begin); - LOG_TEST(log, "Adding header: Range: {}", range_header_value); request.set("Range", range_header_value); } @@ -133,45 +119,7 @@ void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net: credentials.authenticate(request); } -template -std::istream * ReadWriteBufferFromHTTPBase::callImpl( - UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info) -{ - // With empty path poco will send "POST HTTP/1.1" its bug. - if (uri_.getPath().empty()) - uri_.setPath("/"); - - std::optional range; - if (!for_object_info) - { - if (withPartialContent(read_range)) - range = HTTPRange{getOffset(), read_range.end}; - } - - Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - prepareRequest(request, uri_, range); - - LOG_TRACE(log, "Sending request to {}", uri_.toString()); - - auto sess = current_session->getSession(); - auto & stream_out = sess->sendRequest(request); - - if (out_stream_callback) - out_stream_callback(stream_out); - - auto result_istr = receiveResponse(*sess, request, response, true); - response.getCookies(cookies); - - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (!for_object_info) - content_encoding = response.get("Content-Encoding", ""); - - return result_istr; -} - -template -size_t ReadWriteBufferFromHTTPBase::getFileSize() +size_t ReadWriteBufferFromHTTP::getFileSize() { if (!file_info) file_info = getFileInfo(); @@ -179,243 +127,288 @@ size_t ReadWriteBufferFromHTTPBase::getFileSize() if (file_info->file_size) return *file_info->file_size; - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", initial_uri.toString()); } -template -bool ReadWriteBufferFromHTTPBase::supportsReadAt() +bool ReadWriteBufferFromHTTP::supportsReadAt() { if (!file_info) file_info = getFileInfo(); return method == Poco::Net::HTTPRequest::HTTP_GET && file_info->seekable; } -template -bool ReadWriteBufferFromHTTPBase::checkIfActuallySeekable() +bool ReadWriteBufferFromHTTP::checkIfActuallySeekable() { if (!file_info) file_info = getFileInfo(); return file_info->seekable; } -template -String ReadWriteBufferFromHTTPBase::getFileName() const { return uri.toString(); } - -template -void ReadWriteBufferFromHTTPBase::getHeadResponse(Poco::Net::HTTPResponse & response) +String ReadWriteBufferFromHTTP::getFileName() const { - for (size_t i = 0; i < settings.http_max_tries; ++i) - { - try - { - callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD, true, true); - break; - } - catch (const Poco::Exception & e) - { - if (i == settings.http_max_tries - 1 || e.code() == ErrorCodes::TOO_MANY_REDIRECTS || !isRetriableError(response.getStatus())) - throw; - - LOG_ERROR(log, "Failed to make HTTP_HEAD request to {}. Error: {}", uri.toString(), e.displayText()); - } - } + return initial_uri.toString(); } -template -void ReadWriteBufferFromHTTPBase::setupExternalBuffer() +void ReadWriteBufferFromHTTP::getHeadResponse(Poco::Net::HTTPResponse & response) { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not (no hasPendingData() check is needed), - * because this branch means we are prefetching data, - * each nextImpl() call we can fill a different buffer. - */ - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); + doWithRetries( + [&] () + { + callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD, {}); + }, + /*on_retry=*/ nullptr, + /*mute_logging=*/ true); } -template -ReadWriteBufferFromHTTPBase::ReadWriteBufferFromHTTPBase( - UpdatableSessionPtr session_, - Poco::URI uri_, - const Poco::Net::HTTPBasicCredentials & credentials_, +ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( + const HTTPConnectionGroupType & connection_group_, + const Poco::URI & uri_, const std::string & method_, - OutStreamCallback out_stream_callback_, - size_t buffer_size_, - const ReadSettings & settings_, - HTTPHeaderEntries http_header_entries_, + ProxyConfiguration proxy_config_, + ReadSettings read_settings_, + ConnectionTimeouts timeouts_, + const Poco::Net::HTTPBasicCredentials & credentials_, const RemoteHostFilter * remote_host_filter_, - bool delay_initialization, + size_t buffer_size_, + size_t max_redirects_, + OutStreamCallback out_stream_callback_, bool use_external_buffer_, bool http_skip_not_found_url_, - std::optional file_info_, - ProxyConfiguration proxy_config_) + HTTPHeaderEntries http_header_entries_, + bool delay_initialization, + std::optional file_info_) : SeekableReadBuffer(nullptr, 0) - , uri {uri_} - , method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} - , session {session_} - , out_stream_callback {out_stream_callback_} - , credentials {credentials_} - , http_header_entries {std::move(http_header_entries_)} - , remote_host_filter {remote_host_filter_} - , buffer_size {buffer_size_} - , use_external_buffer {use_external_buffer_} - , file_info(file_info_) + , connection_group(connection_group_) + , initial_uri(uri_) + , method(!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET) + , proxy_config(std::move(proxy_config_)) + , read_settings(std::move(read_settings_)) + , timeouts(std::move(timeouts_)) + , credentials(credentials_) + , remote_host_filter(remote_host_filter_) + , buffer_size(buffer_size_) + , max_redirects(max_redirects_) + , use_external_buffer(use_external_buffer_) , http_skip_not_found_url(http_skip_not_found_url_) - , settings {settings_} + , out_stream_callback(std::move(out_stream_callback_)) + , redirects(0) + , http_header_entries {std::move(http_header_entries_)} + , file_info(file_info_) , log(getLogger("ReadWriteBufferFromHTTP")) - , proxy_config(proxy_config_) { - if (settings.http_max_tries <= 0 || settings.http_retry_initial_backoff_ms <= 0 - || settings.http_retry_initial_backoff_ms >= settings.http_retry_max_backoff_ms) + current_uri = initial_uri; + + if (current_uri.getPath().empty()) + current_uri.setPath("/"); + + if (read_settings.http_max_tries <= 0 || read_settings.http_retry_initial_backoff_ms <= 0 + || read_settings.http_retry_initial_backoff_ms >= read_settings.http_retry_max_backoff_ms) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Invalid setting for http backoff, " "must be http_max_tries >= 1 (current is {}) and " "0 < http_retry_initial_backoff_ms < settings.http_retry_max_backoff_ms (now 0 < {} < {})", - settings.http_max_tries, - settings.http_retry_initial_backoff_ms, - settings.http_retry_max_backoff_ms); + read_settings.http_max_tries, + read_settings.http_retry_initial_backoff_ms, + read_settings.http_retry_max_backoff_ms); // Configure User-Agent if it not already set. const std::string user_agent = "User-Agent"; - auto iter = std::find_if( - http_header_entries.begin(), - http_header_entries.end(), - [&user_agent](const HTTPHeaderEntry & entry) { return entry.name == user_agent; }); + auto iter = std::find_if(http_header_entries.begin(), http_header_entries.end(), + [&user_agent] (const HTTPHeaderEntry & entry) { return entry.name == user_agent; }); if (iter == http_header_entries.end()) { - http_header_entries.emplace_back("User-Agent", fmt::format("ClickHouse/{}", VERSION_STRING)); + http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING)); } + if (!delay_initialization && use_external_buffer) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid setting for ReadWriteBufferFromHTTP" + "delay_initialization is false and use_external_buffer it true."); + if (!delay_initialization) { - initialize(); - if (exception) - std::rethrow_exception(exception); + next(); } } -template -void ReadWriteBufferFromHTTPBase::callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors, bool for_object_info) +ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callImpl( + Poco::Net::HTTPResponse & response, const Poco::URI & uri_, const std::string & method_, const std::optional & range, bool allow_redirects) const { - UpdatableSessionPtr current_session = nullptr; + if (remote_host_filter) + remote_host_filter->checkURL(uri_); - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (for_object_info) - current_session = session->clone(uri); - else - current_session = session; + Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, range); - call(current_session, response, method_, throw_on_all_errors, for_object_info); - saved_uri_redirect = uri; + auto session = makeHTTPSession(connection_group, uri_, timeouts, proxy_config); + + auto & stream_out = session->sendRequest(request); + if (out_stream_callback) + out_stream_callback(stream_out); + + auto & resp_stream = session->receiveResponse(response); + + assertResponseIsOk(current_uri.toString(), response, resp_stream, allow_redirects); + + return ReadWriteBufferFromHTTP::CallResult(std::move(session), resp_stream); +} + +ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callWithRedirects( + Poco::Net::HTTPResponse & response, const String & method_, const std::optional & range) +{ + auto result = callImpl(response, current_uri, method_, range, true); while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect = getUriAfterRedirect(*saved_uri_redirect, response); - saved_uri_redirect = uri_redirect; - if (remote_host_filter) - remote_host_filter->checkURL(uri_redirect); + Poco::URI uri_redirect = getUriAfterRedirect(current_uri, response); + ++redirects; + if (redirects > max_redirects) + throw Exception( + ErrorCodes::TOO_MANY_REDIRECTS, + "Too many redirects while trying to access {}." + " You can {} redirects by changing the setting 'max_http_get_redirects'." + " Example: `SET max_http_get_redirects = 10`." + " Redirects are restricted to prevent possible attack when a malicious server redirects to an internal resource, bypassing the authentication or firewall.", + initial_uri.toString(), max_redirects ? "increase the allowed maximum number of" : "allow"); - current_session->updateSession(uri_redirect); + current_uri = uri_redirect; - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - auto result_istr = callImpl(current_session, uri_redirect, response, method, for_object_info); - if (!for_object_info) - istr = result_istr; + result = callImpl(response, uri_redirect, method_, range, true); } + + return result; } -template -void ReadWriteBufferFromHTTPBase::call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors, bool for_object_info) + +void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, + std::function on_retry, + bool mute_logging) const { - try + [[maybe_unused]] auto milliseconds_to_wait = read_settings.http_retry_initial_backoff_ms; + + bool is_retriable = true; + std::exception_ptr exception = nullptr; + + for (size_t attempt = 1; attempt <= read_settings.http_max_tries; ++attempt) { - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - auto result_istr = callImpl(current_session, saved_uri_redirect ? *saved_uri_redirect : uri, response, method_, for_object_info); - if (!for_object_info) - istr = result_istr; - } - catch (...) - { - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (for_object_info) - throw; + [[maybe_unused]] bool last_attempt = attempt + 1 > read_settings.http_max_tries; - if (throw_on_all_errors) - throw; + String error_message; - auto http_status = response.getStatus(); - - if (http_status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND && http_skip_not_found_url) + try { - initialization_error = InitializeError::SKIP_NOT_FOUND_URL; + callable(); + return; } - else if (!isRetriableError(http_status)) + catch (Poco::Net::NetException & e) { - initialization_error = InitializeError::NON_RETRYABLE_ERROR; + error_message = e.displayText(); exception = std::current_exception(); } + catch (DB::NetException & e) + { + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (DB::HTTPException & e) + { + if (!isRetriableError(e.getHTTPStatus())) + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (DB::Exception & e) + { + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (Poco::Exception & e) + { + if (e.code() == POCO_EMFILE) + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + + chassert(exception); + + if (last_attempt || !is_retriable) + { + if (!mute_logging) + LOG_ERROR(log, + "Failed to make request to '{}'. Error: '{}'. " + "Failed at try {}/{}.", + initial_uri.toString(), error_message, + attempt, read_settings.http_max_tries); + + std::rethrow_exception(exception); + } else { - throw; + if (on_retry) + on_retry(); + + if (!mute_logging) + LOG_INFO(log, + "Failed to make request to `{}`. Error: {}. " + "Failed at try {}/{}. " + "Will retry with current backoff wait is {}/{} ms.", + initial_uri.toString(), error_message, + attempt + 1, read_settings.http_max_tries, + milliseconds_to_wait, read_settings.http_retry_max_backoff_ms); + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, read_settings.http_retry_max_backoff_ms); } } } -template -void ReadWriteBufferFromHTTPBase::initialize() + +std::unique_ptr ReadWriteBufferFromHTTP::initialize() { Poco::Net::HTTPResponse response; - call(session, response, method); - if (initialization_error != InitializeError::NONE) - return; + std::optional range; + if (withPartialContent()) + range = HTTPRange{getOffset(), read_range.end}; - while (isRedirect(response.getStatus())) - { - Poco::URI uri_redirect = getUriAfterRedirect(saved_uri_redirect.value_or(uri), response); - if (remote_host_filter) - remote_host_filter->checkURL(uri_redirect); + auto result = callWithRedirects(response, method, range); - session->updateSession(uri_redirect); - - istr = callImpl(session, uri_redirect, response, method); - saved_uri_redirect = uri_redirect; - } - - if (response.hasContentLength()) - LOG_DEBUG(log, "Received response with content length: {}", response.getContentLength()); - - if (withPartialContent(read_range) && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) + if (range.has_value() && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) { /// Having `200 OK` instead of `206 Partial Content` is acceptable in case we retried with range.begin == 0. if (getOffset() != 0) { - if (!exception) + /// Retry 200OK + if (response.getStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) { - exception = std::make_exception_ptr(Exception( + String reason = fmt::format( + "Cannot read with range: [{}, {}] (response status: {}, reason: {}), will retry", + *read_range.begin, read_range.end ? toString(*read_range.end) : "-", + toString(response.getStatus()), response.getReason()); + + /// it is retriable error + throw HTTPException( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + current_uri.toString(), + Poco::Net::HTTPResponse::HTTP_REQUESTED_RANGE_NOT_SATISFIABLE, + reason, + ""); + } + else + throw Exception( ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, "Cannot read with range: [{}, {}] (response status: {}, reason: {})", *read_range.begin, read_range.end ? toString(*read_range.end) : "-", - toString(response.getStatus()), response.getReason())); - } - - /// Retry 200OK - if (response.getStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) - initialization_error = InitializeError::RETRYABLE_ERROR; - else - initialization_error = InitializeError::NON_RETRYABLE_ERROR; - - return; + toString(response.getStatus()), response.getReason()); } else if (read_range.end) { @@ -425,257 +418,140 @@ void ReadWriteBufferFromHTTPBase::initialize() } } + response.getCookies(cookies); + content_encoding = response.get("Content-Encoding", ""); + // Remember file size. It'll be used to report eof in next nextImpl() call. if (!read_range.end && response.hasContentLength()) - file_info = parseFileInfo(response, withPartialContent(read_range) ? getOffset() : 0); + file_info = parseFileInfo(response, range.has_value() ? getOffset() : 0); - impl = std::make_unique(*istr, buffer_size); - - if (use_external_buffer) - setupExternalBuffer(); + return std::move(result).transformToReadBuffer(use_external_buffer ? 0 : buffer_size); } -template -bool ReadWriteBufferFromHTTPBase::nextImpl() +bool ReadWriteBufferFromHTTP::nextImpl() { - if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) - return false; - assert(initialization_error == InitializeError::NONE); - if (next_callback) next_callback(count()); - if ((read_range.end && getOffset() > read_range.end.value()) || - (file_info && file_info->file_size && getOffset() >= file_info->file_size.value())) - { - /// Response was fully read. - markSessionForReuse(session->getSession()); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); - return false; - } + bool next_result = false; - if (impl) - { - if (use_external_buffer) - { - setupExternalBuffer(); - } - else - { - /** - * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read. - */ - if (!working_buffer.empty()) - impl->position() = position(); - } - } - - bool result = false; - size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; - bool last_attempt = false; - - auto on_retriable_error = [&]() - { - retry_with_range_header = true; - impl.reset(); - auto http_session = session->getSession(); - http_session->reset(); - if (!last_attempt) - { - sleepForMilliseconds(milliseconds_to_wait); - milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); - } - }; - - for (size_t i = 0;; ++i) - { - if (last_attempt) - break; - last_attempt = i + 1 >= settings.http_max_tries; - - exception = nullptr; - initialization_error = InitializeError::NONE; - - try + doWithRetries( + /*callable=*/ [&] () { if (!impl) { - initialize(); - - if (initialization_error == InitializeError::NON_RETRYABLE_ERROR) + try { - assert(exception); - break; + impl = initialize(); } - else if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) + catch (HTTPException & e) { - return false; - } - else if (initialization_error == InitializeError::RETRYABLE_ERROR) - { - LOG_TRACE( - log, - "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " - "(Current backoff wait is {}/{} ms)", - uri.toString(), i + 1, settings.http_max_tries, getOffset(), - read_range.end ? toString(*read_range.end) : "unknown", - milliseconds_to_wait, settings.http_retry_max_backoff_ms); + if (http_skip_not_found_url && e.getHTTPStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND) + { + next_result = false; + return; + } - assert(exception); - on_retriable_error(); - continue; + throw; } - assert(!exception); - if (use_external_buffer) { - setupExternalBuffer(); + impl->set(internal_buffer.begin(), internal_buffer.size()); + } + else + { + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); } } - result = impl->next(); - exception = nullptr; - break; - } - catch (const Poco::Exception & e) + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + } + else + { + impl->position() = position(); + } + + next_result = impl->next(); + + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + + offset_from_begin_pos += working_buffer.size(); + }, + /*on_retry=*/ [&] () { - /// Too many open files or redirects - non-retryable. - if (e.code() == POCO_EMFILE || e.code() == ErrorCodes::TOO_MANY_REDIRECTS) - throw; + impl.reset(); + }); - /** Retry request unconditionally if nothing has been read yet. - * Otherwise if it is GET method retry with range header. - */ - bool can_retry_request = !offset_from_begin_pos || method == Poco::Net::HTTPRequest::HTTP_GET; - if (!can_retry_request) - throw; - - LOG_INFO( - log, - "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " - "Error: {}. (Current backoff wait is {}/{} ms)", - uri.toString(), - i + 1, - settings.http_max_tries, - getOffset(), - read_range.end ? toString(*read_range.end) : "unknown", - e.displayText(), - milliseconds_to_wait, - settings.http_retry_max_backoff_ms); - - on_retriable_error(); - exception = std::current_exception(); - } - } - - if (exception) - std::rethrow_exception(exception); - - if (!result) - { - /// Eof is reached, i.e response was fully read. - markSessionForReuse(session->getSession()); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); - return false; - } - - internal_buffer = impl->buffer(); - working_buffer = internal_buffer; - offset_from_begin_pos += working_buffer.size(); - return true; + return next_result; } -template -size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) +size_t ReadWriteBufferFromHTTP::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) const { /// Caller must have checked supportsReadAt(). - /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. + /// This ensures we've sent at least one HTTP request and populated current_uri. chassert(file_info && file_info->seekable); - Poco::URI uri_ = saved_uri_redirect.value_or(uri); - if (uri_.getPath().empty()) - uri_.setPath("/"); - size_t initial_n = n; - size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + size_t total_bytes_copied = 0; + size_t bytes_copied = 0; + bool is_canceled = false; - for (size_t attempt = 0; n > 0; ++attempt) - { - bool last_attempt = attempt + 1 >= settings.http_max_tries; - - Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - prepareRequest(request, uri_, HTTPRange { .begin = offset, .end = offset + n - 1}); - - LOG_TRACE(log, "Sending request to {} for range [{}, {})", uri_.toString(), offset, offset + n); - - auto sess = session->createDetachedSession(uri_); - - Poco::Net::HTTPResponse response; - std::istream * result_istr; - size_t bytes_copied = 0; - - try + doWithRetries( + /*callable=*/ [&] () { - sess->sendRequest(request); - result_istr = receiveResponse(*sess, request, response, /*allow_redirects*/ false); + auto range = HTTPRange{offset, offset + n - 1}; + + Poco::Net::HTTPResponse response; + auto result = callImpl(response, current_uri, method, range, false); if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT && (offset != 0 || offset + n < *file_info->file_size)) - throw Exception( - ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, - "Expected 206 Partial Content, got {} when reading {} range [{}, {})", - toString(response.getStatus()), uri_.toString(), offset, offset + n); - - copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &bytes_copied); - if (bytes_copied == n) { - result_istr->ignore(UINT64_MAX); - /// Response was fully read. - markSessionForReuse(*sess); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); + String reason = fmt::format( + "When reading with readBigAt {}." + "Cannot read with range: [{}, {}] (response status: {}, reason: {}), will retry", + initial_uri.toString(), + *range.begin, *range.end, + toString(response.getStatus()), response.getReason()); + + throw HTTPException( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + current_uri.toString(), + Poco::Net::HTTPResponse::HTTP_REQUESTED_RANGE_NOT_SATISFIABLE, + reason, + ""); } - } - catch (const Poco::Exception & e) + + copyFromIStreamWithProgressCallback(*result.response_stream, to, n, progress_callback, &bytes_copied, &is_canceled); + + offset += bytes_copied; + total_bytes_copied += bytes_copied; + to += bytes_copied; + n -= bytes_copied; + bytes_copied = 0; + }, + /*on_retry=*/ [&] () { - LOG_ERROR( - log, - "HTTP request (positioned) to `{}` with range [{}, {}) failed at try {}/{}: {}", - uri_.toString(), offset, offset + n, attempt + 1, settings.http_max_tries, - e.what()); + offset += bytes_copied; + total_bytes_copied += bytes_copied; + to += bytes_copied; + n -= bytes_copied; + bytes_copied = 0; + }); - /// Decide whether to retry. - - if (last_attempt) - throw; - - /// Too many open files - non-retryable. - if (e.code() == POCO_EMFILE) - throw; - - if (const auto * h = dynamic_cast(&e); - h && !isRetriableError(static_cast(h->getHTTPStatus()))) - throw; - - sleepForMilliseconds(milliseconds_to_wait); - milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); - } - - /// Make sure retries don't re-read the bytes that we've already reported to progress_callback. - offset += bytes_copied; - to += bytes_copied; - n -= bytes_copied; - } - - return initial_n; + chassert(total_bytes_copied == initial_n || is_canceled); + return total_bytes_copied; } -template -off_t ReadWriteBufferFromHTTPBase::getPosition() { return getOffset() - available(); } +off_t ReadWriteBufferFromHTTP::getPosition() +{ + return getOffset() - available(); +} -template -off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int whence) +off_t ReadWriteBufferFromHTTP::seek(off_t offset_, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET mode is allowed."); @@ -688,8 +564,8 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (!working_buffer.empty() && size_t(offset_) >= current_offset - working_buffer.size() && offset_ < current_offset) { pos = working_buffer.end() - (current_offset - offset_); - assert(pos >= working_buffer.begin()); - assert(pos < working_buffer.end()); + chassert(pos >= working_buffer.begin()); + chassert(pos < working_buffer.end()); return getPosition(); } @@ -700,7 +576,7 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (offset_ > position) { size_t diff = offset_ - position; - if (diff < settings.remote_read_min_bytes_for_seek) + if (diff < read_settings.remote_read_min_bytes_for_seek) { ignore(diff); return offset_; @@ -709,6 +585,7 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (!atEndOfRequestedRangeGuess()) ProfileEvents::increment(ProfileEvents::ReadBufferSeekCancelConnection); + impl.reset(); } @@ -719,8 +596,8 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int return offset_; } -template -void ReadWriteBufferFromHTTPBase::setReadUntilPosition(size_t until) + +void ReadWriteBufferFromHTTP::setReadUntilPosition(size_t until) { until = std::max(until, 1ul); if (read_range.end && *read_range.end + 1 == until) @@ -736,8 +613,7 @@ void ReadWriteBufferFromHTTPBase::setReadUntilPosition(size } } -template -void ReadWriteBufferFromHTTPBase::setReadUntilEnd() +void ReadWriteBufferFromHTTP::setReadUntilEnd() { if (!read_range.end) return; @@ -752,11 +628,9 @@ void ReadWriteBufferFromHTTPBase::setReadUntilEnd() } } -template -bool ReadWriteBufferFromHTTPBase::supportsRightBoundedReads() const { return true; } +bool ReadWriteBufferFromHTTP::supportsRightBoundedReads() const { return true; } -template -bool ReadWriteBufferFromHTTPBase::atEndOfRequestedRangeGuess() +bool ReadWriteBufferFromHTTP::atEndOfRequestedRangeGuess() { if (!impl) return true; @@ -767,8 +641,7 @@ bool ReadWriteBufferFromHTTPBase::atEndOfRequestedRangeGues return false; } -template -std::string ReadWriteBufferFromHTTPBase::getResponseCookie(const std::string & name, const std::string & def) const +std::string ReadWriteBufferFromHTTP::getResponseCookie(const std::string & name, const std::string & def) const { for (const auto & cookie : cookies) if (cookie.getName() == name) @@ -776,19 +649,19 @@ std::string ReadWriteBufferFromHTTPBase::getResponseCookie( return def; } -template -void ReadWriteBufferFromHTTPBase::setNextCallback(NextCallback next_callback_) +void ReadWriteBufferFromHTTP::setNextCallback(NextCallback next_callback_) { next_callback = next_callback_; /// Some data maybe already read next_callback(count()); } -template -const std::string & ReadWriteBufferFromHTTPBase::getCompressionMethod() const { return content_encoding; } +const std::string & ReadWriteBufferFromHTTP::getCompressionMethod() const +{ + return content_encoding; +} -template -std::optional ReadWriteBufferFromHTTPBase::tryGetLastModificationTime() +std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { if (!file_info) { @@ -805,12 +678,11 @@ std::optional ReadWriteBufferFromHTTPBase::tryGetLa return file_info->last_modified; } -template -HTTPFileInfo ReadWriteBufferFromHTTPBase::getFileInfo() +ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() { /// May be disabled in case the user knows in advance that the server doesn't support HEAD requests. /// Allows to avoid making unnecessary requests in such cases. - if (!settings.http_make_head_request) + if (!read_settings.http_make_head_request) return HTTPFileInfo{}; Poco::Net::HTTPResponse response; @@ -832,11 +704,11 @@ HTTPFileInfo ReadWriteBufferFromHTTPBase::getFileInfo() throw; } + return parseFileInfo(response, 0); } -template -HTTPFileInfo ReadWriteBufferFromHTTPBase::parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin) +ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin) { HTTPFileInfo res; @@ -869,78 +741,3 @@ HTTPFileInfo ReadWriteBufferFromHTTPBase::parseFileInfo(con } -SessionFactory::SessionFactory(const ConnectionTimeouts & timeouts_, ProxyConfiguration proxy_config_) - : timeouts(timeouts_), proxy_config(proxy_config_) {} - -SessionFactory::SessionType SessionFactory::buildNewSession(const Poco::URI & uri) -{ - return makeHTTPSession(uri, timeouts, proxy_config); -} - -ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, - const ConnectionTimeouts & timeouts, - const Poco::Net::HTTPBasicCredentials & credentials_, - const UInt64 max_redirects, - size_t buffer_size_, - const ReadSettings & settings_, - const HTTPHeaderEntries & http_header_entries_, - const RemoteHostFilter * remote_host_filter_, - bool delay_initialization_, - bool use_external_buffer_, - bool skip_not_found_url_, - std::optional file_info_, - ProxyConfiguration proxy_config_) - : Parent( - std::make_shared(uri_, max_redirects, std::make_shared(timeouts, proxy_config_)), - uri_, - credentials_, - method_, - out_stream_callback_, - buffer_size_, - settings_, - http_header_entries_, - remote_host_filter_, - delay_initialization_, - use_external_buffer_, - skip_not_found_url_, - file_info_, - proxy_config_) {} - - -PooledSessionFactory::PooledSessionFactory( - const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_) - : timeouts(timeouts_) - , per_endpoint_pool_size(per_endpoint_pool_size_) {} - -PooledSessionFactory::SessionType PooledSessionFactory::buildNewSession(const Poco::URI & uri) -{ - return makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size); -} - - -PooledReadWriteBufferFromHTTP::PooledReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, - const Poco::Net::HTTPBasicCredentials & credentials_, - size_t buffer_size_, - const UInt64 max_redirects, - PooledSessionFactoryPtr session_factory) - : Parent( - std::make_shared(uri_, max_redirects, session_factory), - uri_, - credentials_, - method_, - out_stream_callback_, - buffer_size_) {} - - -template class UpdatableSession; -template class UpdatableSession; -template class detail::ReadWriteBufferFromHTTPBase>>; -template class detail::ReadWriteBufferFromHTTPBase>>; - -} diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 63ca3e0417c..5df87fb6149 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -20,7 +20,6 @@ #include #include #include -#include #include #include "config.h" #include @@ -30,44 +29,19 @@ namespace DB { -template -class UpdatableSession +class ReadWriteBufferFromHTTP : public SeekableReadBuffer, public WithFileName, public WithFileSize { public: - using SessionPtr = typename TSessionFactory::SessionType; - - explicit UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr session_factory_); - - SessionPtr getSession(); - - void updateSession(const Poco::URI & uri); - - /// Thread safe. - SessionPtr createDetachedSession(const Poco::URI & uri); - - std::shared_ptr> clone(const Poco::URI & uri); + /// Information from HTTP response header. + struct HTTPFileInfo + { + // nullopt if the server doesn't report it. + std::optional file_size; + std::optional last_modified; + bool seekable = false; + }; private: - SessionPtr session; - UInt64 redirects{0}; - UInt64 max_redirects; - Poco::URI initial_uri; - std::shared_ptr session_factory; -}; - - -/// Information from HTTP response header. -struct HTTPFileInfo -{ - // nullopt if the server doesn't report it. - std::optional file_size; - std::optional last_modified; - bool seekable = false; -}; - - -namespace detail -{ /// Byte range, including right bound [begin, end]. struct HTTPRange { @@ -75,218 +49,208 @@ namespace detail std::optional end; }; - template - class ReadWriteBufferFromHTTPBase : public SeekableReadBuffer, public WithFileName, public WithFileSize + struct CallResult { - protected: - Poco::URI uri; - std::string method; - std::string content_encoding; + HTTPSessionPtr session; + std::istream * response_stream = nullptr; - UpdatableSessionPtr session; - std::istream * istr; /// owned by session - std::unique_ptr impl; - std::function out_stream_callback; - const Poco::Net::HTTPBasicCredentials & credentials; - std::vector cookies; - HTTPHeaderEntries http_header_entries; - const RemoteHostFilter * remote_host_filter = nullptr; - std::function next_callback; + CallResult(HTTPSessionPtr && session_, std::istream & response_stream_) + : session(session_) + , response_stream(&response_stream_) + {} + CallResult(CallResult &&) = default; + CallResult & operator= (CallResult &&) = default; - size_t buffer_size; - bool use_external_buffer; - - size_t offset_from_begin_pos = 0; - HTTPRange read_range; - std::optional file_info; - - /// Delayed exception in case retries with partial content are not satisfiable. - std::exception_ptr exception; - bool retry_with_range_header = false; - /// In case of redirects, save result uri to use it if we retry the request. - std::optional saved_uri_redirect; - - bool http_skip_not_found_url; - - ReadSettings settings; - LoggerPtr log; - - ProxyConfiguration proxy_config; - - bool withPartialContent(const HTTPRange & range) const; - - size_t getOffset() const; - - void prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const; - - std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info = false); - - size_t getFileSize() override; - - bool supportsReadAt() override; - - bool checkIfActuallySeekable() override; - - String getFileName() const override; - - enum class InitializeError - { - RETRYABLE_ERROR, - /// If error is not retriable, `exception` variable must be set. - NON_RETRYABLE_ERROR, - /// Allows to skip not found urls for globs - SKIP_NOT_FOUND_URL, - NONE, - }; - - InitializeError initialization_error = InitializeError::NONE; - - private: - void getHeadResponse(Poco::Net::HTTPResponse & response); - - void setupExternalBuffer(); - - public: - using NextCallback = std::function; - using OutStreamCallback = std::function; - - explicit ReadWriteBufferFromHTTPBase( - UpdatableSessionPtr session_, - Poco::URI uri_, - const Poco::Net::HTTPBasicCredentials & credentials_, - const std::string & method_ = {}, - OutStreamCallback out_stream_callback_ = {}, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - const ReadSettings & settings_ = {}, - HTTPHeaderEntries http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization = false, - bool use_external_buffer_ = false, - bool http_skip_not_found_url_ = false, - std::optional file_info_ = std::nullopt, - ProxyConfiguration proxy_config_ = {}); - - void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false); - - void call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false); - - /** - * Throws if error is retryable, otherwise sets initialization_error = NON_RETRYABLE_ERROR and - * saves exception into `exception` variable. In case url is not found and skip_not_found_url == true, - * sets initialization_error = SKIP_NOT_FOUND_URL, otherwise throws. - */ - void initialize(); - - bool nextImpl() override; - - size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; - - off_t getPosition() override; - - off_t seek(off_t offset_, int whence) override; - - void setReadUntilPosition(size_t until) override; - - void setReadUntilEnd() override; - - bool supportsRightBoundedReads() const override; - - // If true, if we destroy impl now, no work was wasted. Just for metrics. - bool atEndOfRequestedRangeGuess(); - - std::string getResponseCookie(const std::string & name, const std::string & def) const; - - /// Set function to call on each nextImpl, useful when you need to track - /// progress. - /// NOTE: parameter on each call is not incremental -- it's all bytes count - /// passed through the buffer - void setNextCallback(NextCallback next_callback_); - - const std::string & getCompressionMethod() const; - - std::optional tryGetLastModificationTime(); - - HTTPFileInfo getFileInfo(); - - HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); + std::unique_ptr transformToReadBuffer(size_t buf_size) &&; }; -} -class SessionFactory -{ -public: - explicit SessionFactory(const ConnectionTimeouts & timeouts_, ProxyConfiguration proxy_config_ = {}); + const HTTPConnectionGroupType connection_group; + const Poco::URI initial_uri; + const std::string method; + const ProxyConfiguration proxy_config; + const ReadSettings read_settings; + const ConnectionTimeouts timeouts; - using SessionType = HTTPSessionPtr; + const Poco::Net::HTTPBasicCredentials & credentials; + const RemoteHostFilter * remote_host_filter; - SessionType buildNewSession(const Poco::URI & uri); -private: - ConnectionTimeouts timeouts; - ProxyConfiguration proxy_config; -}; + const size_t buffer_size; + const size_t max_redirects; -class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> -{ - using SessionType = UpdatableSession; - using Parent = detail::ReadWriteBufferFromHTTPBase>; + const bool use_external_buffer; + const bool http_skip_not_found_url; + + std::function out_stream_callback; + + Poco::URI current_uri; + size_t redirects = 0; + + std::string content_encoding; + std::unique_ptr impl; + + std::vector cookies; + HTTPHeaderEntries http_header_entries; + std::function next_callback; + + size_t offset_from_begin_pos = 0; + HTTPRange read_range; + std::optional file_info; + + LoggerPtr log; + + bool withPartialContent() const; + + void prepareRequest(Poco::Net::HTTPRequest & request, std::optional range) const; + + void doWithRetries(std::function && callable, std::function on_retry = nullptr, bool mute_logging = false) const; + + CallResult callImpl( + Poco::Net::HTTPResponse & response, + const Poco::URI & uri_, + const std::string & method_, + const std::optional & range, + bool allow_redirects) const; + + CallResult callWithRedirects( + Poco::Net::HTTPResponse & response, + const String & method_, + const std::optional & range); + + std::unique_ptr initialize(); + + size_t getFileSize() override; + + bool supportsReadAt() override; + + bool checkIfActuallySeekable() override; + + String getFileName() const override; + + void getHeadResponse(Poco::Net::HTTPResponse & response); + + void setupExternalBuffer(); + + size_t getOffset() const; + + // If true, if we destroy impl now, no work was wasted. Just for metrics. + bool atEndOfRequestedRangeGuess(); public: + using NextCallback = std::function; + using OutStreamCallback = std::function; + ReadWriteBufferFromHTTP( - Poco::URI uri_, + const HTTPConnectionGroupType & connection_group_, + const Poco::URI & uri_, const std::string & method_, - OutStreamCallback out_stream_callback_, - const ConnectionTimeouts & timeouts, - const Poco::Net::HTTPBasicCredentials & credentials_, - const UInt64 max_redirects = 0, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - const ReadSettings & settings_ = {}, - const HTTPHeaderEntries & http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization_ = true, - bool use_external_buffer_ = false, - bool skip_not_found_url_ = false, - std::optional file_info_ = std::nullopt, - ProxyConfiguration proxy_config_ = {}); -}; - -class PooledSessionFactory -{ -public: - explicit PooledSessionFactory( - const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_); - - using SessionType = PooledHTTPSessionPtr; - - /// Thread safe. - SessionType buildNewSession(const Poco::URI & uri); - -private: - ConnectionTimeouts timeouts; - size_t per_endpoint_pool_size; -}; - -using PooledSessionFactoryPtr = std::shared_ptr; - -class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> -{ - using SessionType = UpdatableSession; - using Parent = detail::ReadWriteBufferFromHTTPBase>; - -public: - explicit PooledReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, + ProxyConfiguration proxy_config_, + ReadSettings read_settings_, + ConnectionTimeouts timeouts_, const Poco::Net::HTTPBasicCredentials & credentials_, + const RemoteHostFilter * remote_host_filter_, size_t buffer_size_, - const UInt64 max_redirects, - PooledSessionFactoryPtr session_factory); + size_t max_redirects_, + OutStreamCallback out_stream_callback_, + bool use_external_buffer_, + bool http_skip_not_found_url_, + HTTPHeaderEntries http_header_entries_, + bool delay_initialization, + std::optional file_info_); + + bool nextImpl() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) const override; + + off_t seek(off_t offset_, int whence) override; + + void setReadUntilPosition(size_t until) override; + + void setReadUntilEnd() override; + + bool supportsRightBoundedReads() const override; + + off_t getPosition() override; + + std::string getResponseCookie(const std::string & name, const std::string & def) const; + + /// Set function to call on each nextImpl, useful when you need to track + /// progress. + /// NOTE: parameter on each call is not incremental -- it's all bytes count + /// passed through the buffer + void setNextCallback(NextCallback next_callback_); + + const std::string & getCompressionMethod() const; + + std::optional tryGetLastModificationTime(); + + HTTPFileInfo getFileInfo(); + static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); }; +using ReadWriteBufferFromHTTPPtr = std::unique_ptr; -extern template class UpdatableSession; -extern template class UpdatableSession; -extern template class detail::ReadWriteBufferFromHTTPBase>>; -extern template class detail::ReadWriteBufferFromHTTPBase>>; +class BuilderRWBufferFromHTTP +{ + Poco::URI uri; + std::string method = Poco::Net::HTTPRequest::HTTP_GET; + HTTPConnectionGroupType connection_group = HTTPConnectionGroupType::HTTP; + ProxyConfiguration proxy_config{}; + ReadSettings read_settings{}; + ConnectionTimeouts timeouts{}; + const RemoteHostFilter * remote_host_filter = nullptr; + size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE; + size_t max_redirects = 0; + ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = nullptr; + bool use_external_buffer = false; + HTTPHeaderEntries http_header_entries{}; + bool delay_initialization = true; + +public: + BuilderRWBufferFromHTTP(Poco::URI uri_) + : uri(uri_) + {} + +#define setterMember(name, member) \ + BuilderRWBufferFromHTTP & name(decltype(BuilderRWBufferFromHTTP::member) arg_##member) \ + { \ + member = std::move(arg_##member); \ + return *this; \ + } + + setterMember(withConnectionGroup, connection_group) + setterMember(withMethod, method) + setterMember(withProxy, proxy_config) + setterMember(withSettings, read_settings) + setterMember(withTimeouts, timeouts) + setterMember(withHostFilter, remote_host_filter) + setterMember(withBufSize, buffer_size) + setterMember(withRedirects, max_redirects) + setterMember(withOutCallback, out_stream_callback) + setterMember(withHeaders, http_header_entries) + setterMember(withExternalBuf, use_external_buffer) + setterMember(withDelayInit, delay_initialization) +#undef setterMember + + ReadWriteBufferFromHTTPPtr create(const Poco::Net::HTTPBasicCredentials & credentials_) + { + return std::make_unique( + connection_group, + uri, + method, + proxy_config, + read_settings, + timeouts, + credentials_, + remote_host_filter, + buffer_size, + max_redirects, + out_stream_callback, + use_external_buffer, + /*http_skip_not_found_url=*/ false, + http_header_entries, + delay_initialization, + /*file_info_=*/ std::nullopt); + } +}; } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index dbb93e63143..a29a4b0b8ee 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,5 +1,4 @@ #include -#include "Common/DNSResolver.h" #include "config.h" #if USE_AWS_S3 @@ -147,9 +146,7 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withTCPKeepAliveTimeout(Poco::Timespan( - client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)) - .withHTTPKeepAliveTimeout(Poco::Timespan( - client_configuration.http_keep_alive_timeout_ms * 1000)); /// flag indicating whether keep-alive is enabled is set to each session upon creation + client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)); } PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration) @@ -164,8 +161,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , get_request_throttler(client_configuration.get_request_throttler) , put_request_throttler(client_configuration.put_request_throttler) , extra_headers(client_configuration.extra_headers) - , http_connection_pool_size(client_configuration.http_connection_pool_size) - , wait_on_pool_size_limit(client_configuration.wait_on_pool_size_limit) { } @@ -308,12 +303,8 @@ void PocoHTTPClient::makeRequestInternal( Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { - /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. const auto request_configuration = per_request_configuration(); - if (http_connection_pool_size) - makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); - else - makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); + makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); } String getMethod(const Aws::Http::HttpRequest & request) @@ -335,7 +326,6 @@ String getMethod(const Aws::Http::HttpRequest & request) } } -template void PocoHTTPClient::makeRequestInternalImpl( Aws::Http::HttpRequest & request, const DB::ProxyConfiguration & proxy_configuration, @@ -343,8 +333,6 @@ void PocoHTTPClient::makeRequestInternalImpl( Aws::Utils::RateLimits::RateLimiterInterface *, Aws::Utils::RateLimits::RateLimiterInterface *) const { - using SessionPtr = std::conditional_t; - LoggerPtr log = getLogger("AWSClient"); auto uri = request.GetUri().GetURIString(); @@ -396,40 +384,17 @@ void PocoHTTPClient::makeRequestInternalImpl( for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt) { Poco::URI target_uri(uri); - SessionPtr session; - if (!proxy_configuration.host.empty()) - { - if (enable_s3_requests_logging) - LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri); - /// Reverse proxy can replace host header with resolved ip address instead of host name. - /// This can lead to request signature difference on S3 side. - if constexpr (pooled) - session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit, - proxy_configuration); - else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - proxy_configuration); - } - else - { - if constexpr (pooled) - session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit); - else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true)); - } + if (enable_s3_requests_logging && !proxy_configuration.isEmpty()) + LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri); + + auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; + + auto session = makeHTTPSession( + group, + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + proxy_configuration); /// In case of error this address will be written to logs request.SetResolvedRemoteHost(session->getResolvedAddress()); @@ -612,10 +577,6 @@ void PocoHTTPClient::makeRequestInternalImpl( response->SetClientErrorMessage(getCurrentExceptionMessage(false)); addMetric(request, S3MetricType::Errors); - - /// Probably this is socket timeout or something more or less related to DNS - /// Let's just remove this host from DNS cache to be more safe - DNSResolver::instance().removeHostFromCache(Poco::URI(uri).getHost()); } } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 5178d75e7b6..a93a4dfbaf7 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include @@ -49,12 +49,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration ThrottlerPtr put_request_throttler; HTTPHeaderEntries extra_headers; - /// Not a client parameter in terms of HTTP and we won't send it to the server. Used internally to determine when connection have to be re-established. - uint32_t http_keep_alive_timeout_ms = 0; - /// Zero means pooling will not be used. - size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit - bool wait_on_pool_size_limit = true; bool s3_use_adaptive_timeouts = true; std::function error_report; @@ -98,12 +93,6 @@ public: ); } - void SetResponseBody(Aws::IStream & incoming_stream, PooledHTTPSessionPtr & session_) /// NOLINT - { - body_stream = Aws::Utils::Stream::ResponseStream( - Aws::New>("http result streambuf", session_, incoming_stream.rdbuf())); - } - void SetResponseBody(std::string & response_body) /// NOLINT { auto stream = Aws::New("http result buf", response_body); // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -163,7 +152,6 @@ private: EnumSize, }; - template void makeRequestInternalImpl( Aws::Http::HttpRequest & request, const DB::ProxyConfiguration & proxy_configuration, @@ -196,9 +184,6 @@ protected: ThrottlerPtr put_request_throttler; const HTTPHeaderEntries extra_headers; - - size_t http_connection_pool_size = 0; - bool wait_on_pool_size_limit = true; }; } diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index c002d30e633..798833e1a9b 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -82,7 +82,7 @@ public: /// (e.g. next() or supportsReadAt()). /// * Performance: there's no buffering. Each readBigAt() call typically translates into actual /// IO operation (e.g. HTTP request). Don't use it for small adjacent reads. - virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) + virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readBigAt() not implemented"); } /// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat). diff --git a/src/IO/S3/SessionAwareIOStream.h b/src/IO/SessionAwareIOStream.h similarity index 97% rename from src/IO/S3/SessionAwareIOStream.h rename to src/IO/SessionAwareIOStream.h index babe52545d1..2380bd0fd60 100644 --- a/src/IO/S3/SessionAwareIOStream.h +++ b/src/IO/SessionAwareIOStream.h @@ -3,7 +3,7 @@ #include -namespace DB::S3 +namespace DB { /** * Wrapper of IOStream to store response stream and corresponding HTTP session. diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 8ddcbc03b84..d54e1685017 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -7,6 +7,7 @@ namespace DB { WriteBufferFromHTTP::WriteBufferFromHTTP( + const HTTPConnectionGroupType & connection_group, const Poco::URI & uri, const std::string & method, const std::string & content_type, @@ -14,9 +15,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const HTTPHeaderEntries & additional_headers, const ConnectionTimeouts & timeouts, size_t buffer_size_, - ProxyConfiguration proxy_configuration) + ProxyConfiguration proxy_configuration +) : WriteBufferFromOStream(buffer_size_) - , session{makeHTTPSession(uri, timeouts, proxy_configuration)} + , session{makeHTTPSession(connection_group, uri, timeouts, proxy_configuration)} , request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} { request.setHost(uri.getHost()); diff --git a/src/IO/WriteBufferFromHTTP.h b/src/IO/WriteBufferFromHTTP.h index f1e1e2a9e91..09fd55ec290 100644 --- a/src/IO/WriteBufferFromHTTP.h +++ b/src/IO/WriteBufferFromHTTP.h @@ -19,7 +19,8 @@ namespace DB class WriteBufferFromHTTP : public WriteBufferFromOStream { public: - explicit WriteBufferFromHTTP(const Poco::URI & uri, + explicit WriteBufferFromHTTP(const HTTPConnectionGroupType & connection_group, + const Poco::URI & uri, const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only const std::string & content_type = "", const std::string & content_encoding = "", diff --git a/src/IO/copyData.cpp b/src/IO/copyData.cpp index 07222a930b5..d2c7200c350 100644 --- a/src/IO/copyData.cpp +++ b/src/IO/copyData.cpp @@ -35,7 +35,7 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t } if (check_bytes && bytes > 0) - throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF."); + throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF, left to copy {} bytes.", bytes); } void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function cancellation_hook, ThrottlerPtr throttler) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a81392cb3d8..d658fbe9920 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -380,8 +380,6 @@ struct ContextSharedPart : boost::noncopyable OrdinaryBackgroundExecutorPtr moves_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); - /// The global pool of HTTP sessions for background fetches. - PooledSessionFactoryPtr fetches_session_factory TSA_GUARDED_BY(background_executors_mutex); RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml HTTPHeaderFilter http_header_filter; /// Forbidden HTTP headers from config.xml @@ -5039,11 +5037,6 @@ void Context::initializeBackgroundExecutorsIfNeeded() ); LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size); - auto timeouts = ConnectionTimeouts::getFetchPartHTTPTimeouts(getServerSettings(), getSettingsRef()); - /// The number of background fetches is limited by the number of threads in the background thread pool. - /// It doesn't make any sense to limit the number of connections per host any further. - shared->fetches_session_factory = std::make_shared(timeouts, background_fetches_pool_size); - shared->fetch_executor = std::make_shared ( "Fetch", @@ -5097,12 +5090,6 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const return shared->common_executor; } -PooledSessionFactoryPtr Context::getCommonFetchesSessionFactory() const -{ - SharedLockGuard lock(shared->background_executors_mutex); - return shared->fetches_session_factory; -} - IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const { callOnce(shared->readers_initialized, [&] { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b2310eaa85d..c8aa3604a6f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -206,9 +206,6 @@ using TemporaryDataOnDiskScopePtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; -class PooledSessionFactory; -using PooledSessionFactoryPtr = std::shared_ptr; - class SessionTracker; struct ServerSettings; @@ -1226,7 +1223,6 @@ public: OrdinaryBackgroundExecutorPtr getMovesExecutor() const; OrdinaryBackgroundExecutorPtr getFetchesExecutor() const; OrdinaryBackgroundExecutorPtr getCommonExecutor() const; - PooledSessionFactoryPtr getCommonFetchesSessionFactory() const; IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; #if USE_LIBURING diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 6a8f82914bf..fe2baea6b4e 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -44,6 +44,11 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const { return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port); } + catch (const DB::NetException &) + { + /// Avoid "Host not found" exceptions + return false; + } catch (const Poco::Net::NetException &) { /// Avoid "Host not found" exceptions diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 37f3c8b2958..026e0c166b4 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -333,10 +334,17 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE); DNSResolver::instance().dropCache(); + HostResolversPool::instance().dropCache(); /// Reinitialize clusters to update their resolved_addresses system_context->reloadClusterConfig(); break; } + case Type::DROP_CONNECTIONS_CACHE: + { + getContext()->checkAccess(AccessType::SYSTEM_DROP_CONNECTIONS_CACHE); + HTTPConnectionPools::instance().dropCache(); + break; + } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context->clearMarkCache(); @@ -1201,6 +1209,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() break; } case Type::DROP_DNS_CACHE: + case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MARK_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index e2ebaee8438..effc7207793 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -384,6 +384,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::KILL: case Type::SHUTDOWN: case Type::DROP_DNS_CACHE: + case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: case Type::DROP_MARK_CACHE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 48be7f6b84f..70a9e27178d 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -22,6 +22,7 @@ public: KILL, SUSPEND, DROP_DNS_CACHE, + DROP_CONNECTIONS_CACHE, DROP_MARK_CACHE, DROP_UNCOMPRESSED_CACHE, DROP_INDEX_MARK_CACHE, diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 8ef2cda5587..2ed55cca30c 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1016,7 +1016,7 @@ private: http_basic_credentials.authenticate(request); } - auto session = makePooledHTTPSession(url, timeouts, 1); + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, url, timeouts); session->sendRequest(request); Poco::Net::HTTPResponse response; @@ -1025,8 +1025,6 @@ private: Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); - /// Response was fully read. - markSessionForReuse(session); auto schema = json_body->getValue("schema"); LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 168c5f729ce..05e1129f9dc 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -526,14 +526,12 @@ std::pair Fetcher::fetchSelected creds.setPassword(password); } - std::unique_ptr in = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - nullptr, - creds, - DBMS_DEFAULT_BUFFER_SIZE, - 0, /* no redirects */ - context->getCommonFetchesSessionFactory()); + auto in = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::HTTP) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(timeouts) + .withDelayInit(false) + .create(creds); int server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); String remote_fs_metadata = parse(in->getResponseCookie("remote_fs_metadata", "")); @@ -557,11 +555,13 @@ std::pair Fetcher::fetchSelected if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { readBinary(sum_files_size, *in); + if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { IMergeTreeDataPart::TTLInfos ttl_infos; String ttl_infos_string; readBinary(ttl_infos_string, *in); + ReadBufferFromString ttl_infos_buffer(ttl_infos_string); assertString("ttl format version: 1\n", ttl_infos_buffer); ttl_infos.read(ttl_infos_buffer); @@ -609,6 +609,7 @@ std::pair Fetcher::fetchSelected } UInt64 revision = parse(in->getResponseCookie("disk_revision", "0")); + if (revision) disk->syncRevision(revision); @@ -743,7 +744,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, bool is_projection, ThrottlerPtr throttler) @@ -799,7 +800,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( void Fetcher::downloadBaseOrProjectionPartToDisk( const String & replica_path, const MutableDataPartStoragePtr & data_part_storage, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler, @@ -807,6 +808,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( { size_t files; readBinary(files, in); + LOG_DEBUG(log, "Downloading files {}", files); + std::vector> written_files; @@ -872,7 +875,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( const String & tmp_prefix, DiskPtr disk, bool to_remote_disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, size_t projections, ThrottlerPtr throttler, diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 8c15dc3cfdb..45a6cf83872 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -20,7 +20,7 @@ namespace DB { class StorageReplicatedMergeTree; -class PooledReadWriteBufferFromHTTP; +class ReadWriteBufferFromHTTP; namespace DataPartsExchange { @@ -94,7 +94,7 @@ private: void downloadBaseOrProjectionPartToDisk( const String & replica_path, const MutableDataPartStoragePtr & data_part_storage, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler, @@ -107,7 +107,7 @@ private: const String & tmp_prefix_, DiskPtr disk, bool to_remote_disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, size_t projections, ThrottlerPtr throttler, @@ -120,7 +120,7 @@ private: const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, bool is_projection, ThrottlerPtr throttler); @@ -131,7 +131,7 @@ private: bool to_detached, const String & tmp_prefix_, DiskPtr disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4e3d8d38b0e..3b766ac8d26 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -963,7 +963,7 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata else columns_to_send = metadata_snapshot->getSampleBlockNonMaterialized().getNames(); - /// DistributedSink will not own cluster, but will own ConnectionPools of the cluster + /// DistributedSink will not own cluster return std::make_shared( local_context, *this, metadata_snapshot, cluster, insert_sync, timeout, StorageID{remote_database, remote_table}, columns_to_send); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 72bbcdd3ea8..11da394feec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1420,8 +1420,10 @@ void StorageS3::Configuration::connect(const ContextPtr & context) url.uri.getScheme()); client_configuration.endpointOverride = url.endpoint; + /// seems as we don't use it client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; + client_configuration.connectTimeoutMs = local_settings.s3_connect_timeout_ms; + auto headers = auth_settings.headers; if (!headers_from_ast.empty()) headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 039be222e7e..b539a152b69 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -461,21 +461,23 @@ std::pair> StorageURLSource: try { auto res = std::make_unique( + HTTPConnectionGroupType::STORAGE, request_uri, http_method, - callback, + proxy_config, + read_settings, timeouts, credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, &context_->getRemoteHostFilter(), + settings.max_read_buffer_size, + settings.max_http_get_redirects, + callback, + /*use_external_buffer*/ false, + skip_url_not_found_error, + headers, delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error, - /* file_info */ std::nullopt, - proxy_config); + /*file_info_*/ std::nullopt); + if (context_->getSettingsRef().engine_url_skip_empty_files && res->eof() && option != std::prev(end)) { @@ -547,7 +549,7 @@ StorageURLSink::StorageURLSink( auto proxy_config = getProxyConfiguration(http_method); auto write_buffer = std::make_unique( - Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config + HTTPConnectionGroupType::STORAGE, Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config ); const auto & settings = context->getSettingsRef(); @@ -1320,24 +1322,17 @@ std::optional IStorageURLBase::tryGetLastModificationTime( auto proxy_config = getProxyConfiguration(uri.getScheme()); - ReadWriteBufferFromHTTP buf( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - {}, - getHTTPTimeouts(context), - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - context->getReadSettings(), - headers, - &context->getRemoteHostFilter(), - true, - false, - false, - std::nullopt, - proxy_config); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(getHTTPTimeouts(context)) + .withHostFilter(&context->getRemoteHostFilter()) + .withBufSize(settings.max_read_buffer_size) + .withRedirects(settings.max_http_get_redirects) + .withHeaders(headers) + .create(credentials); - return buf.tryGetLastModificationTime(); + return buf->tryGetLastModificationTime(); } StorageURL::StorageURL( diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index ca6d40a05a3..a5c16b3a5aa 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -153,17 +153,16 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls)); Poco::Net::HTTPBasicCredentials credentials{}; - ReadWriteBufferFromHTTP buf( - columns_info_uri, - Poco::Net::HTTPRequest::HTTP_POST, - {}, - ConnectionTimeouts::getHTTPTimeouts( - context->getSettingsRef(), - context->getServerSettings().keep_alive_timeout), - credentials); + auto buf = BuilderRWBufferFromHTTP(columns_info_uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(ConnectionTimeouts::getHTTPTimeouts( + context->getSettingsRef(), + context->getServerSettings().keep_alive_timeout)) + .create(credentials); std::string columns_info; - readStringBinary(columns_info, buf); + readStringBinary(columns_info, *buf); NamesAndTypesList columns = NamesAndTypesList::parse(columns_info); return ColumnsDescription{columns}; diff --git a/tests/integration/test_backup_restore_new/test_cancel_backup.py b/tests/integration/test_backup_restore_new/test_cancel_backup.py index 6016bac9197..cce23a7e932 100644 --- a/tests/integration/test_backup_restore_new/test_cancel_backup.py +++ b/tests/integration/test_backup_restore_new/test_cancel_backup.py @@ -177,7 +177,7 @@ def cancel_restore(restore_id): def test_cancel_backup(): # We use partitioning so backups would contain more files. node.query( - "CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%5" + "CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%20" ) node.query(f"INSERT INTO tbl SELECT number FROM numbers(500)") diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml index 23ab57f9330..d94ef68d9c4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -5,6 +5,7 @@ 1 1 + 10000 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 7b1f503ed55..84f7f9f1b6d 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -19,6 +19,7 @@ minio minio123 1 + 10000 diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index e84209a03a1..dbcd7cc3c21 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -278,7 +278,7 @@ def test_unavailable_server(cluster): "Caught exception while loading metadata.*Connection refused" ) assert node2.contains_in_log( - "HTTP request to \`http://nginx:8080/test1/.*\` failed at try 1/10 with bytes read: 0/unknown. Error: Connection refused." + "Failed to make request to 'http://nginx:8080/test1/.*'. Error: 'Connection refused'. Failed at try 10/10." ) finally: node2.exec_in_container( diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index 9c1c9797383..a6db26c8575 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -46,6 +46,7 @@ def cluster_without_dns_cache_update(): except Exception as ex: print(ex) + raise finally: cluster.shutdown() @@ -61,6 +62,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") + node2.query("SYSTEM DROP CONNECTIONS CACHE") # First we check, that normal replication works node1.query( @@ -86,6 +88,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") + node2.query("SYSTEM DROP CONNECTIONS CACHE") # Data is downloaded assert_eq_with_retry(node2, "SELECT count(*) from test_table_drop", "6") @@ -124,6 +127,7 @@ def cluster_with_dns_cache_update(): except Exception as ex: print(ex) + raise finally: cluster.shutdown() @@ -267,6 +271,11 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): privileged=True, user="root", ) + node.exec_in_container( + ["bash", "-c", 'clickhouse client -q "SYSTEM DROP CONNECTIONS CACHE"'], + privileged=True, + user="root", + ) retry_count = 1 assert_eq_with_retry( @@ -296,7 +305,8 @@ def test_host_is_drop_from_cache_after_consecutive_failures( # Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back) # dns_update_short -> dns_max_consecutive_failures set to 6 assert node4.wait_for_log_line( - "Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist." + regexp="Code: 198. DB::NetException: Not found address of host: InvalidHostThatDoesNotExist.", + look_behind_lines=300, ) assert node4.wait_for_log_line( "Cached hosts not found:.*InvalidHostThatDoesNotExist**", diff --git a/tests/integration/test_http_failover/test.py b/tests/integration/test_http_failover/test.py index 41b55ef635c..5920fd980ce 100644 --- a/tests/integration/test_http_failover/test.py +++ b/tests/integration/test_http_failover/test.py @@ -56,9 +56,10 @@ def dst_node_addrs(started_cluster, request): yield - # Clear static DNS entries + # Clear static DNS entries and all keep alive connections src_node.set_hosts([]) src_node.query("SYSTEM DROP DNS CACHE") + src_node.query("SYSTEM DROP CONNECTIONS CACHE") @pytest.mark.parametrize( @@ -77,7 +78,8 @@ def dst_node_addrs(started_cluster, request): def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation): with expectation: result = src_node.query( - "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')", + settings={"http_max_tries": "3"}, ) assert result == "42\n" diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index 17a9a03008e..033f02d7bde 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -150,7 +150,7 @@ def test_url_reconnect(started_cluster): def select(): global result result = node1.query( - "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries = 10, http_retry_max_backoff_ms=1000" + "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries=10, http_retry_max_backoff_ms=1000, http_make_head_request=false" ) assert int(result) == 6581218782194912115 diff --git a/tests/integration/test_s3_table_functions/test.py b/tests/integration/test_s3_table_functions/test.py index a6def175136..ff62d1a9eac 100644 --- a/tests/integration/test_s3_table_functions/test.py +++ b/tests/integration/test_s3_table_functions/test.py @@ -80,6 +80,7 @@ def test_s3_table_functions_timeouts(started_cluster): Test with timeout limit of 1200ms. This should raise an Exception and pass. """ + with PartitionManager() as pm: pm.add_network_delay(node, 1200) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 5ef781bdc9e..70d49b7c1b0 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -118,5 +118,5 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.wfile.write(b"OK") -httpd = http.server.HTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) +httpd = http.server.ThreadingHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) httpd.serve_forever() diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index dc0fdd1a71d..931d18a3f80 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -12,6 +12,7 @@ import urllib.request import subprocess from io import StringIO from http.server import BaseHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn def is_ipv6(host): @@ -145,11 +146,19 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer) t = threading.Thread(target=httpd.serve_forever) return t, httpd diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 7af299c6728..b18ae8a99be 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -105,6 +105,7 @@ NAMED COLLECTION ADMIN ['NAMED COLLECTION CONTROL'] NAMED_COLLECTION ALL SET DEFINER [] USER_NAME ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MMAP CACHE ['SYSTEM DROP MMAP','DROP MMAP CACHE','DROP MMAP'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index f053387d1c5..187f55697e4 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -3,6 +3,7 @@ connect_timeout Seconds 10 connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_secure_ms Milliseconds 3000 external_storage_connect_timeout_sec UInt64 10 +s3_connect_timeout_ms UInt64 1000 filesystem_prefetch_max_memory_usage UInt64 1073741824 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.python b/tests/queries/0_stateless/02205_HTTP_user_agent.python index d8f8a32b6db..83089741bf2 100644 --- a/tests/queries/0_stateless/02205_HTTP_user_agent.python +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.python @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from http.server import SimpleHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn import socket import sys import threading @@ -116,11 +117,19 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(requests_amount): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) def real_func(): for i in range(requests_amount): diff --git a/tests/queries/0_stateless/02233_HTTP_ranged.python b/tests/queries/0_stateless/02233_HTTP_ranged.python index 66ef3304098..5d06e4824b1 100644 --- a/tests/queries/0_stateless/02233_HTTP_ranged.python +++ b/tests/queries/0_stateless/02233_HTTP_ranged.python @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from http.server import BaseHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn import socket import sys import re @@ -206,13 +207,22 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) t = threading.Thread(target=httpd.serve_forever) + t.start() return t, httpd @@ -235,8 +245,6 @@ def run_test(allow_range, settings, check_retries=False): HttpProcessor.responses_to_get = ["500", "200", "206"] retries_num = len(HttpProcessor.responses_to_get) - t, httpd = start_server() - t.start() test_select(settings) download_buffer_size = settings["max_download_buffer_size"] @@ -261,12 +269,12 @@ def run_test(allow_range, settings, check_retries=False): if HttpProcessor.range_used: raise Exception("HTTP Range used while not supported") - httpd.shutdown() - t.join() print("PASSED") def main(): + t, httpd = start_server() + settings = {"max_download_buffer_size": 20} # Test Accept-Ranges=False @@ -285,10 +293,15 @@ def main(): settings["max_download_threads"] = 2 run_test(allow_range=True, settings=settings, check_retries=True) + httpd.shutdown() + t.join() + if __name__ == "__main__": try: main() + sys.stdout.flush() + os._exit(0) except Exception as ex: exc_type, exc_value, exc_traceback = sys.exc_info() traceback.print_tb(exc_traceback, file=sys.stderr) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index 288f1129b53..e346d9893a7 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "INSERT TO S3" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | sort +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort echo "CHECK WITH query_log" $CLICKHOUSE_CLIENT -nq " diff --git a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh index ce90157d004..e4a1de9a2ec 100755 --- a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh +++ b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh @@ -19,8 +19,8 @@ query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -nm --query " WITH - ProfileEvents['ReadBufferFromS3ResetSessions'] AS reset, - ProfileEvents['ReadBufferFromS3PreservedSessions'] AS preserved + ProfileEvents['DiskConnectionsReset'] AS reset, + ProfileEvents['DiskConnectionsPreserved'] AS preserved SELECT preserved > reset FROM system.query_log WHERE type = 'QueryFinish' @@ -51,7 +51,7 @@ select queryID() from( " 2>&1) ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -nm --query " -SELECT ProfileEvents['ReadWriteBufferFromHTTPPreservedSessions'] > 0 +SELECT ProfileEvents['StorageConnectionsPreserved'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() diff --git a/tests/queries/0_stateless/02833_url_without_path_encoding.sh b/tests/queries/0_stateless/02833_url_without_path_encoding.sh index b71586099cf..eb845c6b45b 100755 --- a/tests/queries/0_stateless/02833_url_without_path_encoding.sh +++ b/tests/queries/0_stateless/02833_url_without_path_encoding.sh @@ -8,5 +8,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=1" # Grep 'test%2Fa.tsv' to ensure that path wasn't encoded/decoded -$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | grep -o "test%2Fa.tsv" -m1 - +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | \ + grep -o "test%2Fa.tsv" -m1 | head -n 1 From f7f1d86e667117dd51b74747c1b1ed5f9339c466 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 4 Mar 2024 14:37:17 +0100 Subject: [PATCH 55/66] fix tests test_attach_without_fetching test_replicated_merge_tree_wait_on_shutdown --- src/Common/CurrentMetrics.cpp | 2 +- src/Disks/IO/ReadBufferFromWebServer.cpp | 1 - tests/integration/test_attach_without_fetching/test.py | 1 + .../test_replicated_merge_tree_wait_on_shutdown/test.py | 4 ++++ 4 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index f43481f665b..dfbf6199361 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -274,7 +274,7 @@ M(DistrCacheUsedConnections, "Number of currently used connections to Distributed Cache") \ M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ - M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") + M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") \ \ M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index 7509aa81d75..03300cc0714 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -114,7 +114,6 @@ bool ReadBufferFromWebServer::nextImpl() chassert(working_buffer.begin() != nullptr); chassert(impl->buffer().begin() != nullptr); - chassert(working_buffer.begin() == impl->buffer().begin()); chassert(impl->available() == 0); diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index b430387e0f1..67352e2dcbe 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -56,6 +56,7 @@ def check_data(nodes, detached_parts): node.query_with_retry("SYSTEM SYNC REPLICA test") + for node in nodes: print("> Checking data integrity for", node.name) for i in range(10): diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 67dd03098e9..995afedf415 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -40,6 +40,10 @@ def test_shutdown_and_wait(start_cluster): f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()" ) + # we stop merges on node1 to make node2 fetch all 51 origin parts from node1 + # and not to fetch a smaller set of merged covering parts + node1.query("SYSTEM STOP MERGES test_table") + node1.query("INSERT INTO test_table VALUES (0)") node2.query("SYSTEM SYNC REPLICA test_table") From 4df406d3adce0ae1fb55d742cf59ddd928e96ddb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Mar 2024 13:56:51 +0100 Subject: [PATCH 56/66] work with review notes --- src/Client/Connection.cpp | 9 ++++---- src/Common/HTTPConnectionPool.cpp | 2 +- src/Common/HostResolvePool.cpp | 13 +++++++---- src/Common/HostResolvePool.h | 2 -- src/Common/ProfileEvents.cpp | 6 ++--- src/Common/tests/gtest_connection_pool.cpp | 27 ++++++++++++++++++++++ 6 files changed, 45 insertions(+), 14 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a11a1243957..180942e6b83 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -212,8 +212,9 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); + /// Add server address to exception. Exception will preserve stack trace. + e.addMessage("({})", getDescription()); + throw; } catch (Poco::Net::NetException & e) { @@ -222,7 +223,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + /// Add server address to exception. Also Exception will remember new stack trace. It's a pity that more precise exception type is lost. throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); } catch (Poco::TimeoutException & e) @@ -232,7 +233,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + /// Add server address to exception. Also Exception will remember new stack trace. It's a pity that more precise exception type is lost. /// This exception can only be thrown from socket->connect(), so add information about connection timeout. const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; throw NetException( diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 18ffef34091..a21438a11a2 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -336,7 +336,7 @@ private: request_stream_completed = false; response_stream = nullptr; - response_stream_completed = true; + response_stream_completed = false; return result; } diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp index f6cc9c919ba..6db28edc07e 100644 --- a/src/Common/HostResolvePool.cpp +++ b/src/Common/HostResolvePool.cpp @@ -13,7 +13,7 @@ namespace ProfileEvents { extern const Event AddressesDiscovered; extern const Event AddressesExpired; - extern const Event AddressesFailScored; + extern const Event AddressesMarkedAsFailed; } namespace CurrentMetrics @@ -34,7 +34,7 @@ HostResolverMetrics HostResolver::getMetrics() return HostResolverMetrics{ .discovered = ProfileEvents::AddressesDiscovered, .expired = ProfileEvents::AddressesExpired, - .failed = ProfileEvents::AddressesFailScored, + .failed = ProfileEvents::AddressesMarkedAsFailed, .active_count = CurrentMetrics::AddressesActive, }; } @@ -120,7 +120,6 @@ void HostResolver::updateWeights() } chassert((getTotalWeight() > 0 && !records.empty()) || records.empty()); - random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); } HostResolver::Entry HostResolver::resolve() @@ -170,6 +169,7 @@ void HostResolver::setFail(const Poco::Net::IPAddress & address) Poco::Net::IPAddress HostResolver::selectBest() { chassert(!records.empty()); + auto random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); size_t weight = random_weight_picker(thread_local_rng); auto it = std::partition_point(records.begin(), records.end(), [&](const Record & rec) { return rec.weight_prefix_sum <= weight; }); chassert(it != records.end()); @@ -178,8 +178,13 @@ Poco::Net::IPAddress HostResolver::selectBest() HostResolver::Records::iterator HostResolver::find(const Poco::Net::IPAddress & addr) TSA_REQUIRES(mutex) { - return std::lower_bound( + auto it = std::lower_bound( records.begin(), records.end(), addr, [](const Record & rec, const Poco::Net::IPAddress & value) { return rec.address < value; }); + + if (it != records.end() && it->address != addr) + return records.end(); + + return it; } bool HostResolver::isUpdateNeeded() diff --git a/src/Common/HostResolvePool.h b/src/Common/HostResolvePool.h index 2a31cec3b2d..4f127f05253 100644 --- a/src/Common/HostResolvePool.h +++ b/src/Common/HostResolvePool.h @@ -191,8 +191,6 @@ protected: Poco::Timestamp last_resolve_time TSA_GUARDED_BY(mutex); Records records TSA_GUARDED_BY(mutex); - std::uniform_int_distribution random_weight_picker TSA_GUARDED_BY(mutex); - Poco::Logger * log = &Poco::Logger::get("ConnectionPool"); }; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0c9582ab4fb..c1ac3d08245 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -721,9 +721,9 @@ The server successfully detected this situation and will download merged part fr M(HTTPConnectionsErrors, "Number of cases when creation of a http connection failed") \ M(HTTPConnectionsElapsedMicroseconds, "Total time spend on creating http connections") \ \ - M(AddressesDiscovered, "Total count of new addresses in dns resolve results for connection pools") \ - M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for for connection pools") \ - M(AddressesFailScored, "Total count of new addresses in dns resolve results for for connection pools") \ + M(AddressesDiscovered, "Total count of new addresses in dns resolve results for http connections") \ + M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for http connections") \ + M(AddressesMarkedAsFailed, "Total count of addresses which has been marked as faulty due to connection errors for http connections") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp index 01b78958442..c271cc0e2ec 100644 --- a/src/Common/tests/gtest_connection_pool.cpp +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -552,6 +552,33 @@ TEST_F(ConnectionPoolTest, HardLimit) ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} + +TEST_F(ConnectionPoolTest, NoReceiveCall) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + + { + auto data = String("Hello"); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_PUT, "/", "HTTP/1.1"); // HTTP/1.1 is required for keep alive + request.setContentLength(data.size()); + std::ostream & ostream = connection->sendRequest(request); + ostream << data; + } + + connection->flushRequest(); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); From 21b04143e81e5e2e9a6c5fa02103bcafdb4a27ed Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Mar 2024 19:17:25 +0100 Subject: [PATCH 57/66] set vat RECORDS in main proccess --- tests/queries/0_stateless/02998_system_dns_cache_table.sh | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02998_system_dns_cache_table.sh b/tests/queries/0_stateless/02998_system_dns_cache_table.sh index 41d2386fe9c..b74fc00ab3b 100755 --- a/tests/queries/0_stateless/02998_system_dns_cache_table.sh +++ b/tests/queries/0_stateless/02998_system_dns_cache_table.sh @@ -5,12 +5,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Retries are necessary because the DNS cache may be flushed before second statement is executed -i=0 retries=3 +i=0 +retries=5 while [[ $i -lt $retries ]]; do - ${CLICKHOUSE_CURL} -sS --fail --data "SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers())" "${CLICKHOUSE_URL}" | grep -oP -q 'Ok.' && \ + ${CLICKHOUSE_CURL} -sS --fail --data "SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers())" "${CLICKHOUSE_URL}" | grep -oP -q 'Ok.' || continue + RECORDS=$(${CLICKHOUSE_CURL} -sS --fail --data "SELECT hostname, ip_address, ip_family, (isNotNull(cached_at) AND cached_at > '1970-01-01 00:00:00') FROM system.dns_cache WHERE hostname = 'localhost' and ip_family = 'IPv4';" "${CLICKHOUSE_URL}") - if [ "${RECORDS}" != "" ]; then + if [[ -n "${RECORDS}" ]]; then echo "${RECORDS}" exit 0 fi From 77c5de700f62451c6f2cf55620d9522832b5d56b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 9 Mar 2024 13:35:12 +0100 Subject: [PATCH 58/66] fix how web disk read empty directories --- .../ObjectStorages/Web/WebObjectStorage.cpp | 5 +-- src/IO/ReadWriteBufferFromHTTP.h | 4 ++- src/Storages/StorageURL.cpp | 32 ++++++++----------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 0bad668a404..4adb92cf5c8 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -53,6 +53,7 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lockgetReadSettings()) .withTimeouts(timeouts) .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSkipNotFound(true) .create(credentials); String file_name; @@ -98,10 +99,6 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock> StorageURLSource: try { - auto res = std::make_unique( - HTTPConnectionGroupType::STORAGE, - request_uri, - http_method, - proxy_config, - read_settings, - timeouts, - credentials, - &context_->getRemoteHostFilter(), - settings.max_read_buffer_size, - settings.max_http_get_redirects, - callback, - /*use_external_buffer*/ false, - skip_url_not_found_error, - headers, - delay_initialization, - /*file_info_*/ std::nullopt); - + auto res = BuilderRWBufferFromHTTP(request_uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(http_method) + .withProxy(proxy_config) + .withSettings(read_settings) + .withTimeouts(timeouts) + .withHostFilter(&context_->getRemoteHostFilter()) + .withBufSize(settings.max_read_buffer_size) + .withRedirects(settings.max_http_get_redirects) + .withOutCallback(callback) + .withSkipNotFound(skip_url_not_found_error) + .withHeaders(headers) + .withDelayInit(delay_initialization) + .create(credentials); if (context_->getSettingsRef().engine_url_skip_empty_files && res->eof() && option != std::prev(end)) { From f429f54af8f1fc8fad56a96818c3fe6cc8a1ee7a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sat, 9 Mar 2024 20:38:23 +0000 Subject: [PATCH 59/66] CI: make style check fast #do_not_test --- tests/ci/style_check.py | 32 +++++++---- .../{check_cpp_docs.sh => check_cpp.sh} | 55 ++++++++++++------- utils/check-style/check_py.sh | 19 ++++--- 3 files changed, 68 insertions(+), 38 deletions(-) rename utils/check-style/{check_cpp_docs.sh => check_cpp.sh} (50%) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index d0565e136d3..a772539aef8 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,7 +10,7 @@ from pathlib import Path from typing import List, Tuple from docker_images_helper import get_docker_image, pull_image -from env_helper import REPO_COPY, TEMP_PATH +from env_helper import CI, REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -128,32 +128,40 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - # pr_info = PRInfo() + pr_info = PRInfo() IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd_1 = ( + cmd_cpp = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_cpp_docs.sh" + f"{image} ./check_cpp.sh" ) - cmd_2 = ( + cmd_py = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " f"{image} ./check_py.sh" ) - logging.info("Is going to run the command: %s", cmd_1) - logging.info("Is going to run the command: %s", cmd_2) with ProcessPoolExecutor(max_workers=2) as executor: - # Submit commands for execution in parallel - future1 = executor.submit(subprocess.run, cmd_1, shell=True) - future2 = executor.submit(subprocess.run, cmd_2, shell=True) - # Wait for both commands to complete + logging.info("Is going to run the command: %s", cmd_cpp) + future1 = executor.submit(subprocess.run, cmd_cpp, shell=True) + # Parallelization does not make it faster - run subsequently _ = future1.result() - _ = future2.result() + + run_pycheck = True + if CI and pr_info.number > 0: + # skip py check if PR and no changed py files + pr_info.fetch_changed_files() + if not any(file.endswith(".py") for file in pr_info.changed_files): + run_pycheck = False + + if run_pycheck: + logging.info("Is going to run the command: %s", cmd_py) + future2 = executor.submit(subprocess.run, cmd_py, shell=True) + _ = future2.result() # if args.push: # checkout_head(pr_info) diff --git a/utils/check-style/check_cpp_docs.sh b/utils/check-style/check_cpp.sh similarity index 50% rename from utils/check-style/check_cpp_docs.sh rename to utils/check-style/check_cpp.sh index 7ad3cede758..20d4c7f5ee9 100755 --- a/utils/check-style/check_cpp_docs.sh +++ b/utils/check-style/check_cpp.sh @@ -4,31 +4,48 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv -# FIXME: 30 sec to wait -# echo "Check duplicates" | ts -# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +start_total=`date +%s` -echo "Check style" | ts +# 40 sec - too much +# start=`date +%s` +# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +# runtime=$((`date +%s`-start)) +# echo "Duplicates check. Done. $runtime seconds." + +start=`date +%s` ./check-style -n |& tee /test_output/style_output.txt -echo "Check typos" | ts -./check-typos |& tee /test_output/typos_output.txt -echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/docs_spelling_output.txt -echo "Check whitespaces" | ts +runtime=$((`date +%s`-start)) +echo "Check style. Done. $runtime seconds." + +start=`date +%s` ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt -echo "Check workflows" | ts +runtime=$((`date +%s`-start)) +echo "Check whitespaces. Done. $runtime seconds." + +start=`date +%s` ./check-workflows |& tee /test_output/workflows_output.txt -echo "Check submodules" | ts +runtime=$((`date +%s`-start)) +echo "Check workflows. Done. $runtime seconds." + +start=`date +%s` ./check-submodules |& tee /test_output/submodules_output.txt -echo "Check style. Done" | ts +runtime=$((`date +%s`-start)) +echo "Check submodules. Done. $runtime seconds." + +start=`date +%s` +./check-typos |& tee /test_output/typos_output.txt +runtime=$((`date +%s`-start)) +echo "Check typos. Done. $runtime seconds." + +start=`date +%s` +./check-doc-aspell |& tee /test_output/docs_spelling_output.txt +runtime=$((`date +%s`-start)) +echo "Check docs spelling. Done. $runtime seconds." + +runtime=$((`date +%s`-start_total)) +echo "Check style, total. Done. $runtime seconds." + # FIXME: 6 min to wait # echo "Check shell scripts with shellcheck" | ts # ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt - - -# FIXME: move out -# /process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv -# echo "Check help for changelog generator works" | ts -# cd ../changelog || exit 1 -# ./changelog.py -h 2>/dev/null 1>&2 diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 48c02013734..5caeffa65fe 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -1,17 +1,22 @@ #!/bin/bash -# yaml check is not the best one - cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv # FIXME: 1 min to wait + head checkout # echo "Check python formatting with black" | ts # ./check-black -n |& tee /test_output/black_output.txt -echo "Check pylint" | ts -./check-pylint -n |& tee /test_output/pylint_output.txt -echo "Check pylint. Done" | ts +start_total=`date +%s` -echo "Check python type hinting with mypy" | ts +start=`date +%s` +./check-pylint -n |& tee /test_output/pylint_output.txt +runtime=$((`date +%s`-start)) +echo "Check pylint. Done. $runtime seconds." + +start=`date +%s` ./check-mypy -n |& tee /test_output/mypy_output.txt -echo "Check python type hinting with mypy. Done" | ts +runtime=$((`date +%s`-start)) +echo "Check python type hinting with mypy. Done. $runtime seconds." + +runtime=$((`date +%s`-start_total)) +echo "Check python total. Done. $runtime seconds." From 8c08fc1f579861d02690653cc05a87826e59a5b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Mar 2024 04:09:26 +0300 Subject: [PATCH 60/66] Revert "CI: make style check faster" --- tests/ci/style_check.py | 32 ++++------- .../{check_cpp.sh => check_cpp_docs.sh} | 55 +++++++------------ utils/check-style/check_py.sh | 17 ++---- 3 files changed, 37 insertions(+), 67 deletions(-) rename utils/check-style/{check_cpp.sh => check_cpp_docs.sh} (50%) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index a772539aef8..d0565e136d3 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,7 +10,7 @@ from pathlib import Path from typing import List, Tuple from docker_images_helper import get_docker_image, pull_image -from env_helper import CI, REPO_COPY, TEMP_PATH +from env_helper import REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -128,40 +128,32 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - pr_info = PRInfo() + # pr_info = PRInfo() IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd_cpp = ( + cmd_1 = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_cpp.sh" + f"{image} ./check_cpp_docs.sh" ) - cmd_py = ( + cmd_2 = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " f"{image} ./check_py.sh" ) + logging.info("Is going to run the command: %s", cmd_1) + logging.info("Is going to run the command: %s", cmd_2) with ProcessPoolExecutor(max_workers=2) as executor: - logging.info("Is going to run the command: %s", cmd_cpp) - future1 = executor.submit(subprocess.run, cmd_cpp, shell=True) - # Parallelization does not make it faster - run subsequently + # Submit commands for execution in parallel + future1 = executor.submit(subprocess.run, cmd_1, shell=True) + future2 = executor.submit(subprocess.run, cmd_2, shell=True) + # Wait for both commands to complete _ = future1.result() - - run_pycheck = True - if CI and pr_info.number > 0: - # skip py check if PR and no changed py files - pr_info.fetch_changed_files() - if not any(file.endswith(".py") for file in pr_info.changed_files): - run_pycheck = False - - if run_pycheck: - logging.info("Is going to run the command: %s", cmd_py) - future2 = executor.submit(subprocess.run, cmd_py, shell=True) - _ = future2.result() + _ = future2.result() # if args.push: # checkout_head(pr_info) diff --git a/utils/check-style/check_cpp.sh b/utils/check-style/check_cpp_docs.sh similarity index 50% rename from utils/check-style/check_cpp.sh rename to utils/check-style/check_cpp_docs.sh index 20d4c7f5ee9..7ad3cede758 100755 --- a/utils/check-style/check_cpp.sh +++ b/utils/check-style/check_cpp_docs.sh @@ -4,48 +4,31 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv -start_total=`date +%s` - -# 40 sec - too much -# start=`date +%s` +# FIXME: 30 sec to wait +# echo "Check duplicates" | ts # ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt -# runtime=$((`date +%s`-start)) -# echo "Duplicates check. Done. $runtime seconds." -start=`date +%s` +echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt -runtime=$((`date +%s`-start)) -echo "Check style. Done. $runtime seconds." - -start=`date +%s` -./check-whitespaces -n |& tee /test_output/whitespaces_output.txt -runtime=$((`date +%s`-start)) -echo "Check whitespaces. Done. $runtime seconds." - -start=`date +%s` -./check-workflows |& tee /test_output/workflows_output.txt -runtime=$((`date +%s`-start)) -echo "Check workflows. Done. $runtime seconds." - -start=`date +%s` -./check-submodules |& tee /test_output/submodules_output.txt -runtime=$((`date +%s`-start)) -echo "Check submodules. Done. $runtime seconds." - -start=`date +%s` +echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt -runtime=$((`date +%s`-start)) -echo "Check typos. Done. $runtime seconds." - -start=`date +%s` +echo "Check docs spelling" | ts ./check-doc-aspell |& tee /test_output/docs_spelling_output.txt -runtime=$((`date +%s`-start)) -echo "Check docs spelling. Done. $runtime seconds." - -runtime=$((`date +%s`-start_total)) -echo "Check style, total. Done. $runtime seconds." - +echo "Check whitespaces" | ts +./check-whitespaces -n |& tee /test_output/whitespaces_output.txt +echo "Check workflows" | ts +./check-workflows |& tee /test_output/workflows_output.txt +echo "Check submodules" | ts +./check-submodules |& tee /test_output/submodules_output.txt +echo "Check style. Done" | ts # FIXME: 6 min to wait # echo "Check shell scripts with shellcheck" | ts # ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt + + +# FIXME: move out +# /process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv +# echo "Check help for changelog generator works" | ts +# cd ../changelog || exit 1 +# ./changelog.py -h 2>/dev/null 1>&2 diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 5caeffa65fe..48c02013734 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -1,22 +1,17 @@ #!/bin/bash +# yaml check is not the best one + cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv # FIXME: 1 min to wait + head checkout # echo "Check python formatting with black" | ts # ./check-black -n |& tee /test_output/black_output.txt -start_total=`date +%s` - -start=`date +%s` +echo "Check pylint" | ts ./check-pylint -n |& tee /test_output/pylint_output.txt -runtime=$((`date +%s`-start)) -echo "Check pylint. Done. $runtime seconds." +echo "Check pylint. Done" | ts -start=`date +%s` +echo "Check python type hinting with mypy" | ts ./check-mypy -n |& tee /test_output/mypy_output.txt -runtime=$((`date +%s`-start)) -echo "Check python type hinting with mypy. Done. $runtime seconds." - -runtime=$((`date +%s`-start_total)) -echo "Check python total. Done. $runtime seconds." +echo "Check python type hinting with mypy. Done" | ts From d16c6780c7b5c6b7180e0744c55856b1e5b8285d Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Thu, 25 Jan 2024 11:50:18 -0400 Subject: [PATCH 61/66] Adds basic emptyArrayUIntX docs. --- .../functions/array-functions.md | 174 +++++++++++++++++- 1 file changed, 172 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 2120b675c73..6dea8e89b6a 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -19,7 +19,7 @@ empty([x]) An array is considered empty if it does not contain any elements. :::note -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. +Can be optimized by enabling the [`optimize_functions_to_subcolumns` setting](../../operations/settings/settings.md#optimize-functions-to-subcolumns). With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. ::: The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). @@ -104,7 +104,177 @@ Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operat Alias: `OCTET_LENGTH` -## emptyArrayUInt8, emptyArrayUInt16, emptyArrayUInt32, emptyArrayUInt64 +## emptyArrayUInt8 + +Accepts zero arguments and returns an empty UInt8 array. + +**Syntax** + +```sql +emptyArrayUInt8() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt8)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt8()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 255) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1234 Alex 33 [] +1234 Alex 33 [255] +``` + +## emptyArrayUInt16 + +Accepts zero arguments and returns an empty UInt16 array. + +**Syntax** + +```sql +emptyArrayUInt16() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt16)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt16()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 65535) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [65535] +``` + +## emptyArrayUInt32 + +Accepts zero arguments and returns an empty UInt32 array. + +**Syntax** + +```sql +emptyArrayUInt32() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt32)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt32()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 4294967295) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [4294967295] +``` + +## emptyArrayUInt64 + +Accepts zero arguments and returns an empty UInt64 array. + +**Syntax** + +```sql +emptyArrayUInt64() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt64)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt64()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 18446744073709551615) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [18446744073709551615] +``` ## emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64 From 515cb59b00ac358a95f65ab7b76b8349bc6e02e7 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Thu, 25 Jan 2024 12:29:23 -0400 Subject: [PATCH 62/66] Adds emptyArrayIntX docs. --- .../functions/array-functions.md | 172 +++++++++++++++++- 1 file changed, 171 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 6dea8e89b6a..ebeac4196d7 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -276,7 +276,177 @@ Result: 1231 Alex 33 [18446744073709551615] ``` -## emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64 +## emptyArrayInt8 + +Accepts zero arguments and returns an empty Int8 array. + +**Syntax** + +```sql +emptyArrayInt8() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int8)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt8()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 127) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1234 Alex 33 [] +1234 Alex 33 [127] +``` + +## emptyArrayInt16 + +Accepts zero arguments and returns an empty Int16 array. + +**Syntax** + +```sql +emptyArrayInt16() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int16)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt16()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 32767) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [32767] +``` + +## emptyArrayInt32 + +Accepts zero arguments and returns an empty Int32 array. + +**Syntax** + +```sql +emptyArrayInt32() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int32)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt32()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 2147483647) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [2147483647] +``` + +## emptyArrayInt64 + +Accepts zero arguments and returns an empty Int64 array. + +**Syntax** + +```sql +emptyArrayInt64() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int64)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt64()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 9223372036854775807) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [9223372036854775807] +``` ## emptyArrayFloat32, emptyArrayFloat64 From c3895c3546b66d3d1b8eac0e3e9377d4da9a1f29 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Thu, 25 Jan 2024 14:41:32 -0400 Subject: [PATCH 63/66] Adds emptyArrayFloat32/64 docs. --- .../functions/array-functions.md | 86 ++++++++++++++++++- 1 file changed, 85 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index ebeac4196d7..471dd0ff3f5 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -448,7 +448,91 @@ Result: 1231 Alex 33 [9223372036854775807] ``` -## emptyArrayFloat32, emptyArrayFloat64 +## emptyArrayFloat32 + +Returns an empty Float32 array. Accepts zero arguments. + +**Syntax** + +```sql +emptyArrayFloat32() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int32)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayFloat32()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 3.4e+38) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [-2147483648] +``` + +## emptyArrayFloat64 + +Accepts zero arguments and returns an empty Float64 array. + +**Syntax** + +```sql +emptyArrayFloat64() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayFloat64()); +SELECT * FROM users; + +ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 1.999999999e+9) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 [1999999999] +``` ## emptyArrayDate, emptyArrayDateTime From ade0f9f75fcf0d8e7394655a1938a16ed86c85a1 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 29 Jan 2024 14:21:58 -0400 Subject: [PATCH 64/66] Adds docs for emptyArrayDate and emptyArrayDateTime. --- .../functions/array-functions.md | 123 +++++++++++++++++- 1 file changed, 122 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 471dd0ff3f5..d811112a861 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -536,9 +536,130 @@ Result: ## emptyArrayDate, emptyArrayDateTime +## emptyArrayDate + +Accepts zero arguments and returns an empty Date array. + +**Syntax** + +```sql +emptyArrayDate() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, userDates Array(Date)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayDate()); +SELECT * FROM users; + +ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, today()) WHERE uid = 1231; +ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, addDays(today(), 1)) WHERE uid = 1231; +SELECT * FROM users; +``` + +## emptyArrayDateTime + +Accepts zero arguments and returns an empty DateTime array. + +**Syntax** + +```sql +emptyArrayDateTime() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, userDates Array(DateTime)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayDateTime()); +SELECT * FROM users; + +ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, now()) WHERE uid = 1231; +ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, addDays(now(), 1)) WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 ['2024-01-29 18:08:01','2024-01-30 18:08:01'] +``` + ## emptyArrayString -Accepts zero arguments and returns an empty array of the appropriate type. +Accepts zero arguments and returns an empty String array. + +**Syntax** + +```sql +emptyArrayString() +``` + +**Arguments** + +None. + +**Returned value** + +An empty array. + +**Implementation details** + +None. + +**Examples** + +Query: + +```sql +CREATE TABLE users (uid Int16, name String, age Int16, aliases Array(String)) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayString()); +SELECT * FROM users; + +ALTER TABLE users UPDATE aliases = arrayPushBack(aliases, 'Al') WHERE uid = 1231; +ALTER TABLE users UPDATE aliases = arrayPushBack(aliases, 'Little Alex') WHERE uid = 1231; +SELECT * FROM users; +``` + +Result: + +```response +1231 Alex 33 [] +1231 Alex 33 ['Al','Little Alex'] +``` ## emptyArrayToSingle From e7270069a487be62e067e1f158ad0d745280622b Mon Sep 17 00:00:00 2001 From: Johnny <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 29 Jan 2024 14:26:20 -0400 Subject: [PATCH 65/66] Removes dupliated h2 header. --- docs/en/sql-reference/functions/array-functions.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index d811112a861..d3054d25106 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -534,8 +534,6 @@ Result: 1231 Alex 33 [1999999999] ``` -## emptyArrayDate, emptyArrayDateTime - ## emptyArrayDate Accepts zero arguments and returns an empty Date array. From 3c60e829710ffb72c8571ef664ad146cfcd3f266 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Mar 2024 07:39:08 +0000 Subject: [PATCH 66/66] Incorporate review feedback --- .../functions/array-functions.md | 222 +++--------------- 1 file changed, 39 insertions(+), 183 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index d3054d25106..67a4c026851 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -106,7 +106,7 @@ Alias: `OCTET_LENGTH` ## emptyArrayUInt8 -Accepts zero arguments and returns an empty UInt8 array. +Returns an empty UInt8 array. **Syntax** @@ -122,34 +122,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt8)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt8()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 255) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayUInt8(); ``` Result: ```response -1234 Alex 33 [] -1234 Alex 33 [255] +[] ``` ## emptyArrayUInt16 -Accepts zero arguments and returns an empty UInt16 array. +Returns an empty UInt16 array. **Syntax** @@ -165,34 +154,24 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt16)) ENGINE=Memory; +SELECT emptyArrayUInt16(); -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt16()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 65535) WHERE uid = 1231; -SELECT * FROM users; ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [65535] +[] ``` ## emptyArrayUInt32 -Accepts zero arguments and returns an empty UInt32 array. +Returns an empty UInt32 array. **Syntax** @@ -208,34 +187,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt32)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt32()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 4294967295) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayUInt32(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [4294967295] +[] ``` ## emptyArrayUInt64 -Accepts zero arguments and returns an empty UInt64 array. +Returns an empty UInt64 array. **Syntax** @@ -251,34 +219,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(UInt64)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayUInt64()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 18446744073709551615) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayUInt64(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [18446744073709551615] +[] ``` ## emptyArrayInt8 -Accepts zero arguments and returns an empty Int8 array. +Returns an empty Int8 array. **Syntax** @@ -294,34 +251,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int8)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt8()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 127) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayInt8(); ``` Result: ```response -1234 Alex 33 [] -1234 Alex 33 [127] +[] ``` ## emptyArrayInt16 -Accepts zero arguments and returns an empty Int16 array. +Returns an empty Int16 array. **Syntax** @@ -337,34 +283,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int16)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt16()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 32767) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayInt16(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [32767] +[] ``` ## emptyArrayInt32 -Accepts zero arguments and returns an empty Int32 array. +Returns an empty Int32 array. **Syntax** @@ -380,34 +315,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int32)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt32()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 2147483647) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayInt32(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [2147483647] +[] ``` ## emptyArrayInt64 -Accepts zero arguments and returns an empty Int64 array. +Returns an empty Int64 array. **Syntax** @@ -423,34 +347,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int64)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayInt64()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 9223372036854775807) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayInt64(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [9223372036854775807] +[] ``` ## emptyArrayFloat32 -Returns an empty Float32 array. Accepts zero arguments. +Returns an empty Float32 array. **Syntax** @@ -466,34 +379,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int32)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayFloat32()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 3.4e+38) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayFloat32(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [-2147483648] +[] ``` ## emptyArrayFloat64 -Accepts zero arguments and returns an empty Float64 array. +Returns an empty Float64 array. **Syntax** @@ -509,34 +411,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, contacts Array(Int)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayFloat64()); -SELECT * FROM users; - -ALTER TABLE users UPDATE contacts = arrayPushBack(contacts, 1.999999999e+9) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayFloat64(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 [1999999999] +[] ``` ## emptyArrayDate -Accepts zero arguments and returns an empty Date array. +Returns an empty Date array. **Syntax** @@ -552,33 +443,22 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, userDates Array(Date)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayDate()); -SELECT * FROM users; - -ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, today()) WHERE uid = 1231; -ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, addDays(today(), 1)) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayDate(); ``` ## emptyArrayDateTime -Accepts zero arguments and returns an empty DateTime array. +Returns an empty DateTime array. **Syntax** ```sql -emptyArrayDateTime() +[] ``` **Arguments** @@ -589,35 +469,23 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, userDates Array(DateTime)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayDateTime()); -SELECT * FROM users; - -ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, now()) WHERE uid = 1231; -ALTER TABLE users UPDATE userDates = arrayPushBack(userDates, addDays(now(), 1)) WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayDateTime(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 ['2024-01-29 18:08:01','2024-01-30 18:08:01'] +[] ``` ## emptyArrayString -Accepts zero arguments and returns an empty String array. +Returns an empty String array. **Syntax** @@ -633,30 +501,18 @@ None. An empty array. -**Implementation details** - -None. - **Examples** Query: ```sql -CREATE TABLE users (uid Int16, name String, age Int16, aliases Array(String)) ENGINE=Memory; - -INSERT INTO users VALUES (1231, 'Alex', 33, emptyArrayString()); -SELECT * FROM users; - -ALTER TABLE users UPDATE aliases = arrayPushBack(aliases, 'Al') WHERE uid = 1231; -ALTER TABLE users UPDATE aliases = arrayPushBack(aliases, 'Little Alex') WHERE uid = 1231; -SELECT * FROM users; +SELECT emptyArrayString(); ``` Result: ```response -1231 Alex 33 [] -1231 Alex 33 ['Al','Little Alex'] +[] ``` ## emptyArrayToSingle