From 83022b77714a204ef4025d0b5081fbc127f2a586 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 2 Sep 2023 21:56:36 +0200 Subject: [PATCH 001/105] Added support for parameterized view with analyzer by analyzing the select part with default values --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 47 +++++++++++++++++++++ src/Analyzer/TableFunctionNode.cpp | 7 +++ src/Analyzer/TableFunctionNode.h | 3 ++ src/Interpreters/InterpreterCreateQuery.cpp | 38 +++++++++++++++-- 4 files changed, 92 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 348189854e8..c82d3079118 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -77,6 +77,12 @@ #include #include #include +#include +#include +#include +#include +#include +#include namespace ProfileEvents { @@ -6210,8 +6216,49 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, QueryExpressionsAliasVisitor & expressions_visitor, bool nested_table_function) { + + String database_name = scope.context->getCurrentDatabase(); + String table_name = table_function_node->getOriginalAST()->as()->name; + + if (table_function_node->getOriginalAST()->as()->is_compound_name) + { + std::vector parts; + splitInto<'.'>(parts, table_function_node->getOriginalAST()->as()->name); + + if (parts.size() == 2) + { + database_name = parts[0]; + table_name = parts[1]; + } + } + auto & table_function_node_typed = table_function_node->as(); + StoragePtr table = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, scope.context->getQueryContext()); + if (table) + { + if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) + { + auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_function_node->getOriginalAST()); + StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); + + ASTCreateQuery create; + create.select = query->as(); + auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, scope.context); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); + res->startup(); + table_function_node->getOriginalAST()->as()->prefer_subquery_to_function_formatting = true; + table_function_node_typed.resolve(std::move(res), scope.context); + return; + } + } + + if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index e5158a06373..f4ffe7f4ee5 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -36,6 +36,13 @@ void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePt unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_); } +void TableFunctionNode::resolve(StoragePtr storage_value, ContextPtr context) +{ + storage = std::move(storage_value); + storage_id = storage->getStorageID(); + storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); +} + const StorageID & TableFunctionNode::getStorageID() const { if (!storage) diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 69237ac8416..ed1a26c4dd4 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -100,6 +100,9 @@ public: /// Resolve table function with table function, storage and context void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_); + /// Resolve table function as parameterized view with storage and context + void resolve(StoragePtr storage_value, ContextPtr context); + /// Get storage id, throws exception if function node is not resolved const StorageID & getStorageID() const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 92d74f4f18a..58b6722aae9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -76,7 +76,8 @@ #include #include - +#include +#include namespace DB { @@ -745,12 +746,43 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } else if (create.select) { - Block as_select_sample; if (getContext()->getSettingsRef().allow_experimental_analyzer) { - as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + if (create.isParameterizedView()) + { + auto select = create.select->clone(); + + ///Get all query parameters + const auto parameters = analyzeReceiveQueryParamsWithType(select); + NameToNameMap parameter_values; + + for (const auto & parameter : parameters) + { + const auto data_type = DataTypeFactory::instance().get(parameter.second); + /// Todo improve getting default values & include more datatypes + if (data_type->isValueRepresentedByNumber() || parameter.second == "String") + parameter_values[parameter.first] = "1"; + else if (parameter.second.starts_with("Array") || parameter.second.starts_with("Map")) + parameter_values[parameter.first] = "[]"; + else + parameter_values[parameter.first] = " "; + LOG_INFO(&Poco::Logger::get("InterpreterCreateQuery"), "parameter = {} = {} ", parameter.first, parameter_values[parameter.first]); + + } + + /// Replace with default parameters + ReplaceQueryParameterVisitor visitor(parameter_values); + visitor.visit(select); + + as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(select, getContext()); + } + else + { + as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + } else { From 2dfda84da0e16c594df7df4eb2b05ee1baba1193 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 2 Sep 2023 21:57:57 +0200 Subject: [PATCH 002/105] Removed parameterized view tests from analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 15d46403da9..5521234495f 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -88,7 +88,6 @@ 02402_merge_engine_with_view 02404_memory_bound_merging 02426_orc_bug -02428_parameterized_view 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number @@ -123,7 +122,6 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long -02818_parameterized_view_with_cte_multiple_usage 02790_optimize_skip_unused_shards_join 01940_custom_tld_sharding_key 02815_range_dict_no_direct_join From 59195e1199d5c8ed31f4243b58f3186771219295 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 4 Sep 2023 19:03:23 +0200 Subject: [PATCH 003/105] Removed log for each parameter --- src/Interpreters/InterpreterCreateQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 58b6722aae9..66c219dcd56 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -768,8 +768,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti parameter_values[parameter.first] = "[]"; else parameter_values[parameter.first] = " "; - LOG_INFO(&Poco::Logger::get("InterpreterCreateQuery"), "parameter = {} = {} ", parameter.first, parameter_values[parameter.first]); - } /// Replace with default parameters From 96c4b6bc35ee818afd2d2963dec7afdb5583969c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 20 Nov 2023 14:41:14 +0100 Subject: [PATCH 004/105] Updated to not analyze create parameterized view for analyzer & old analyzer --- src/Interpreters/InterpreterCreateQuery.cpp | 48 +++++-------------- src/Storages/StorageView.cpp | 3 +- .../0_stateless/02428_parameterized_view.sh | 2 +- 3 files changed, 14 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 747c0be009e..4ee666e2a9a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -649,6 +649,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (!attach && !is_restore_from_backup && context_->getSettingsRef().flatten_nested) res.flattenNested(); + if (res.getAllPhysical().empty()) throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED, "Cannot CREATE table without physical columns"); @@ -755,49 +756,22 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti { Block as_select_sample; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + if (!create.isParameterizedView()) { - if (create.isParameterizedView()) - { - auto select = create.select->clone(); - - ///Get all query parameters - const auto parameters = analyzeReceiveQueryParamsWithType(select); - NameToNameMap parameter_values; - - for (const auto & parameter : parameters) - { - const auto data_type = DataTypeFactory::instance().get(parameter.second); - /// Todo improve getting default values & include more datatypes - if (data_type->isValueRepresentedByNumber() || parameter.second == "String") - parameter_values[parameter.first] = "1"; - else if (parameter.second.starts_with("Array") || parameter.second.starts_with("Map")) - parameter_values[parameter.first] = "[]"; - else - parameter_values[parameter.first] = " "; - } - - /// Replace with default parameters - ReplaceQueryParameterVisitor visitor(parameter_values); - visitor.visit(select); - - as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(select, getContext()); - } - else + if (getContext()->getSettingsRef().allow_experimental_analyzer) { as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); } + else + { + as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), + getContext(), + false /* is_subquery */, + create.isParameterizedView()); + } + properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } - else - { - as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), - getContext(), - false /* is_subquery */, - create.isParameterizedView()); - } - - properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } else if (create.as_table_function) { diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index f0f9b9540de..2f7267e3701 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -112,7 +112,8 @@ StorageView::StorageView( : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + if (is_parameterized_view_ && !query.isParameterizedView()) + storage_metadata.setColumns(columns_); storage_metadata.setComment(comment); if (!query.select) diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index ad9c672f4c5..499b8697ffc 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -37,7 +37,7 @@ $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv1 AS SELECT * FROM test_02428_Ca $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv1(price=20)" $CLICKHOUSE_CLIENT -q "SELECT Price FROM \`test_02428_pv1\`(price=20)" -$CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv1" 2>&1 | grep -Fq "UNKNOWN_QUERY_PARAMETER" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv1" 2>&1 | grep -q "UNKNOWN_QUERY_PARAMETER\|UNKNOWN_IDENTIFIER" && echo 'ERROR' || echo 'OK' $CLICKHOUSE_CLIENT --param_p 10 -q "SELECT Price FROM test_02428_pv1(price={p:UInt64})" $CLICKHOUSE_CLIENT --param_l 1 -q "SELECT Price FROM test_02428_pv1(price=50) LIMIT ({l:UInt64})" From 356fc0aadb8f7c0f15f72c3b72955e1db7046e48 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 7 Jan 2024 14:49:24 +0100 Subject: [PATCH 005/105] Fix tests --- src/Storages/StorageView.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 2f7267e3701..1898e49de86 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -112,8 +112,14 @@ StorageView::StorageView( : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; - if (is_parameterized_view_ && !query.isParameterizedView()) + if (is_parameterized_view_) + { + if (!query.isParameterizedView()) + storage_metadata.setColumns(columns_); + } + else storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); if (!query.select) From 605c76e66ea5bdd2644026a5c7425e87f24c3702 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 16 Jan 2024 11:22:27 +0100 Subject: [PATCH 006/105] Fix test fails --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 0dc3026afc0..b235918c438 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6400,23 +6400,27 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, { String database_name = scope.context->getCurrentDatabase(); - String table_name = table_function_node->getOriginalAST()->as()->name; + String table_name = ""; - if (table_function_node->getOriginalAST()->as()->is_compound_name) + if (table_function_node->getOriginalAST() && table_function_node->getOriginalAST()->as()) { - std::vector parts; - splitInto<'.'>(parts, table_function_node->getOriginalAST()->as()->name); - - if (parts.size() == 2) + table_name = table_function_node->getOriginalAST()->as()->name; + if (table_function_node->getOriginalAST()->as()->is_compound_name) { - database_name = parts[0]; - table_name = parts[1]; + std::vector parts; + splitInto<'.'>(parts, table_function_node->getOriginalAST()->as()->name); + + if (parts.size() == 2) + { + database_name = parts[0]; + table_name = parts[1]; + } } } auto & table_function_node_typed = table_function_node->as(); - StoragePtr table = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, scope.context->getQueryContext()); + StoragePtr table = table_name.empty() ? nullptr : DatabaseCatalog::instance().tryGetTable({database_name, table_name}, scope.context->getQueryContext()); if (table) { if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) From 8d88f4cf87d13c6760a5235abf4180102daf8b5c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 17 Jan 2024 09:42:53 +0100 Subject: [PATCH 007/105] Update setting is_parameterized_view & settings columns for view --- src/Interpreters/InterpreterCreateQuery.cpp | 5 ++++- src/Storages/StorageView.cpp | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7985785aa9f..6031c8b4e46 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -780,8 +780,11 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.constraints = as_storage_metadata->getConstraints(); } - else if (create.select && !create.isParameterizedView()) + else if (create.select) { + if (create.isParameterizedView()) + return properties; + Block as_select_sample; if (getContext()->getSettingsRef().allow_experimental_analyzer) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 1898e49de86..6b80e2450c4 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -112,7 +112,7 @@ StorageView::StorageView( : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; - if (is_parameterized_view_) + if (!is_parameterized_view_) { if (!query.isParameterizedView()) storage_metadata.setColumns(columns_); From d3b4dea8058e1cccb34bf39b3f26b4c0e5b2368a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 17 Jan 2024 20:02:17 +0100 Subject: [PATCH 008/105] Fix clang tidy build --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 6 +----- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index d2270ea9910..7322d53d831 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6401,7 +6401,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, { String database_name = scope.context->getCurrentDatabase(); - String table_name = ""; + String table_name; if (table_function_node->getOriginalAST() && table_function_node->getOriginalAST()->as()) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6031c8b4e46..e71946caafe 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -809,11 +809,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti * for example: LIMIT, OFFSET, functions parameters, functions constant only arguments. */ - SelectQueryOptions options; - if (create.isParameterizedView()) - options = options.createParameterizedView(); - - InterpreterSelectWithUnionQuery interpreter(create.select->clone(), getContext(), options); + InterpreterSelectWithUnionQuery interpreter(create.select->clone(), getContext(), SelectQueryOptions()); as_select_sample = interpreter.getSampleBlock(); } From df0c018a9be06e9ccbfb40460f29b155aa86b57f Mon Sep 17 00:00:00 2001 From: Hongbin Ma Date: Fri, 12 Jan 2024 16:09:09 +0800 Subject: [PATCH 009/105] support T64 for date32 type --- src/Compression/CompressionCodecT64.cpp | 6 +++++ .../00873_t64_codec_date.reference | 4 +++ .../0_stateless/00873_t64_codec_date.sql | 26 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/00873_t64_codec_date.reference create mode 100644 tests/queries/0_stateless/00873_t64_codec_date.sql diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index bf9a9414bc1..42c6a18aa77 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -91,6 +91,7 @@ enum class MagicNumber : uint8_t Decimal32 = 19, Decimal64 = 20, IPv4 = 21, + Date32 = 22, }; MagicNumber serializeTypeId(std::optional type_id) @@ -109,6 +110,7 @@ MagicNumber serializeTypeId(std::optional type_id) case TypeIndex::Int32: return MagicNumber::Int32; case TypeIndex::Int64: return MagicNumber::Int64; case TypeIndex::Date: return MagicNumber::Date; + case TypeIndex::Date32: return MagicNumber::Date32; case TypeIndex::DateTime: return MagicNumber::DateTime; case TypeIndex::DateTime64: return MagicNumber::DateTime64; case TypeIndex::Enum8: return MagicNumber::Enum8; @@ -137,6 +139,7 @@ TypeIndex deserializeTypeId(uint8_t serialized_type_id) case MagicNumber::Int32: return TypeIndex::Int32; case MagicNumber::Int64: return TypeIndex::Int64; case MagicNumber::Date: return TypeIndex::Date; + case MagicNumber::Date32: return TypeIndex::Date32; case MagicNumber::DateTime: return TypeIndex::DateTime; case MagicNumber::DateTime64: return TypeIndex::DateTime64; case MagicNumber::Enum8: return TypeIndex::Enum8; @@ -177,6 +180,8 @@ TypeIndex baseType(TypeIndex type_idx) case TypeIndex::Enum16: case TypeIndex::Date: return TypeIndex::UInt16; + case TypeIndex::Date32: + return TypeIndex::Int32; case TypeIndex::UInt32: case TypeIndex::DateTime: case TypeIndex::IPv4: @@ -205,6 +210,7 @@ TypeIndex typeIdx(const IDataType * data_type) case TypeIndex::UInt16: case TypeIndex::Enum16: case TypeIndex::Date: + case TypeIndex::Date32: case TypeIndex::Int32: case TypeIndex::UInt32: case TypeIndex::IPv4: diff --git a/tests/queries/0_stateless/00873_t64_codec_date.reference b/tests/queries/0_stateless/00873_t64_codec_date.reference new file mode 100644 index 00000000000..1568c3122e6 --- /dev/null +++ b/tests/queries/0_stateless/00873_t64_codec_date.reference @@ -0,0 +1,4 @@ +1970-01-01 1970-01-01 1950-01-01 1950-01-01 +1970-01-01 1970-01-01 1970-01-01 1970-01-01 +2149-06-06 2149-06-06 2149-06-08 2149-06-08 +2149-06-06 2149-06-06 2149-06-06 2149-06-06 diff --git a/tests/queries/0_stateless/00873_t64_codec_date.sql b/tests/queries/0_stateless/00873_t64_codec_date.sql new file mode 100644 index 00000000000..e9230c75665 --- /dev/null +++ b/tests/queries/0_stateless/00873_t64_codec_date.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS t64; + +CREATE TABLE t64 +( + date16 Date, + t_date16 Date Codec(T64, ZSTD), + date_32 Date32, + t_date32 Date32 Codec(T64, ZSTD) +) ENGINE MergeTree() ORDER BY tuple(); + +INSERT INTO t64 values ('1970-01-01', '1970-01-01', '1970-01-01', '1970-01-01'); +INSERT INTO t64 values ('2149-06-06', '2149-06-06', '2149-06-06', '2149-06-06'); +INSERT INTO t64 values ('2149-06-08', '2149-06-08', '2149-06-08', '2149-06-08'); +INSERT INTO t64 values ('1950-01-01', '1950-01-01', '1950-01-01', '1950-01-01'); + +SELECT * FROM t64 ORDER BY date16; + +SELECT * FROM t64 WHERE date16 != t_date16; +SELECT * FROM t64 WHERE date_32 != t_date32; + +OPTIMIZE TABLE t64 FINAL; + +SELECT * FROM t64 WHERE date16 != t_date16; +SELECT * FROM t64 WHERE date_32 != t_date32; + +DROP TABLE t64; From 36055bd0089f52473f893d71c475a2782a45e8b4 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 25 Jan 2024 21:44:46 +0000 Subject: [PATCH 010/105] init --- src/Functions/FunctionBinaryArithmetic.h | 106 ++++++++++++++---- src/Functions/IsOperation.h | 4 +- .../02975_intdiv_with_decimal.reference | 52 +++++++++ .../0_stateless/02975_intdiv_with_decimal.sql | 54 +++++++++ 4 files changed, 196 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02975_intdiv_with_decimal.reference create mode 100644 tests/queries/0_stateless/02975_intdiv_with_decimal.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 1b2519d1ec5..e34514d15fd 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -146,10 +146,24 @@ private: /// it's not correct for Decimal public: static constexpr bool allow_decimal = IsOperation::allow_decimal; + static constexpr bool only_integer = IsOperation::div_int || IsOperation::div_int_or_zero; /// Appropriate result type for binary operator on numeric types. "Date" can also mean /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). using ResultDataType = Switch< + /// Result must be Integer + Case< + only_integer && IsDataTypeDecimal && IsDataTypeDecimal, + Switch< + Case || std::is_same_v, DataTypeInt256>, + Case || std::is_same_v, DataTypeInt128>, + Case || std::is_same_v, DataTypeInt64>, + Case || std::is_same_v, DataTypeInt32>>>, + Case< + only_integer, + Switch< + Case, LeftDataType>, + Case, RightDataType>>>, /// Decimal cases Case || IsDataTypeDecimal), InvalidType>, Case< @@ -1667,31 +1681,77 @@ public: { if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { - if constexpr (is_division) + if constexpr (is_div_int || is_div_int_or_zero) { - if (context->getSettingsRef().decimal_check_overflow) - { - /// Check overflow by using operands scale (based on big decimal division implementation details): - /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers - /// i.e. int_operand = decimal_operand*10^scale - /// For division, left operand will be scaled by right operand scale also to do big integer division, - /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale - /// So, we can check upfront possible overflow just by checking max scale used for left operand - /// Note: it doesn't detect all possible overflow during big decimal division - if (left.getScale() + right.getScale() > ResultDataType::maxPrecision()) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); - } + if constexpr (std::is_same_v || std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v || std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v || std::is_same_v) + type_res = std::make_shared(); + else + type_res = std::make_shared(); + } + else + { + if constexpr (is_division) + { + if (context->getSettingsRef().decimal_check_overflow) + { + /// Check overflow by using operands scale (based on big decimal division implementation details): + /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers + /// i.e. int_operand = decimal_operand*10^scale + /// For division, left operand will be scaled by right operand scale also to do big integer division, + /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale + /// So, we can check upfront possible overflow just by checking max scale used for left operand + /// Note: it doesn't detect all possible overflow during big decimal division + if (left.getScale() + right.getScale() > ResultDataType::maxPrecision()) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); + } + } + ResultDataType result_type = decimalResultType(left, right); + type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } - ResultDataType result_type = decimalResultType(left, right); - type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } else if constexpr ((IsDataTypeDecimal && IsFloatingPoint) || (IsDataTypeDecimal && IsFloatingPoint)) type_res = std::make_shared(); else if constexpr (IsDataTypeDecimal) - type_res = std::make_shared(left.getPrecision(), left.getScale()); + { + if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegral) + type_res = std::make_shared(); + else if constexpr (is_div_int || is_div_int_or_zero) + { + if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else + type_res = std::make_shared(); + } + else + type_res = std::make_shared(left.getPrecision(), left.getScale()); + } else if constexpr (IsDataTypeDecimal) - type_res = std::make_shared(right.getPrecision(), right.getScale()); + { + if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegral) + type_res = std::make_shared(); + else if constexpr (is_div_int || is_div_int_or_zero) + { + if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else + type_res = std::make_shared(); + } + else + type_res = std::make_shared(right.getPrecision(), right.getScale()); + } else if constexpr (std::is_same_v) { // Special case for DateTime: binary OPS should reuse timezone @@ -2009,8 +2069,10 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A constexpr bool decimal_with_float = (IsDataTypeDecimal && IsFloatingPoint) || (IsFloatingPoint && IsDataTypeDecimal); - using T0 = std::conditional_t; - using T1 = std::conditional_t; + constexpr bool is_div_int_with_decimal = (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal); + + using T0 = std::conditional_t>; + using T1 = std::conditional_t>; using ResultType = typename ResultDataType::FieldType; using ColVecT0 = ColumnVectorOrDecimal; using ColVecT1 = ColumnVectorOrDecimal; @@ -2026,6 +2088,12 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A left_col = castColumn(arguments[0], converted_type); right_col = castColumn(arguments[1], converted_type); } + else if constexpr (is_div_int_with_decimal) + { + const auto converted_type = std::make_shared(); + left_col = castColumn(arguments[0], converted_type); + right_col = castColumn(arguments[1], converted_type); + } else { left_col = arguments[0].column; diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 8ea53c865ce..b36530591ef 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -62,7 +62,9 @@ struct IsOperation static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; - static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; + static constexpr bool division_allow_decimal = div_floating || modulo; + + static constexpr bool allow_decimal = plus || minus || multiply || division_allow_decimal || least || greatest; }; } diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.reference b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference new file mode 100644 index 00000000000..9c1faab21d7 --- /dev/null +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference @@ -0,0 +1,52 @@ +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.sql b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql new file mode 100644 index 00000000000..8fc4b5a9a7d --- /dev/null +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql @@ -0,0 +1,54 @@ +--intDiv-- +SELECT intDiv(4,2); +SELECT intDiv(toDecimal32(4.4, 2), 2); +SELECT intDiv(4, toDecimal32(2.2, 2)); +SELECT intDiv(toDecimal32(4.4, 2), 2); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal32(2.2, 2)); +SELECT intDiv(toDecimal64(4.4, 3), 2); +SELECT intDiv(toDecimal64(4.4, 3), toDecimal32(2.2, 2)); +SELECT intDiv(toDecimal128(4.4, 4), 2); +SELECT intDiv(toDecimal128(4.4, 4), toDecimal32(2.2, 2)); +SELECT intDiv(toDecimal256(4.4, 5), 2); +SELECT intDiv(toDecimal256(4.4, 5), toDecimal32(2.2, 2)); +SELECT intDiv(4, toDecimal64(2.2, 2)); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDiv(4, toDecimal128(2.2, 3)); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal128(2.2, 3)); +SELECT intDiv(4, toDecimal256(2.2, 4)); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal256(2.2, 4)); +SELECT intDiv(toDecimal64(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDiv(toDecimal128(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDiv(toDecimal256(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDiv(toDecimal64(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDiv(toDecimal128(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDiv(toDecimal256(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDiv(toDecimal64(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDiv(toDecimal128(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDiv(toDecimal256(4.4, 2), toDecimal256(2.2, 2)); +--intDivOrZero-- +SELECT intDivOrZero(4,2); +SELECT intDivOrZero(toDecimal32(4.4, 2), 2); +SELECT intDivOrZero(4, toDecimal32(2.2, 2)); +SELECT intDivOrZero(toDecimal32(4.4, 2), 2); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal32(2.2, 2)); +SELECT intDivOrZero(toDecimal64(4.4, 3), 2); +SELECT intDivOrZero(toDecimal64(4.4, 3), toDecimal32(2.2, 2)); +SELECT intDivOrZero(toDecimal128(4.4, 4), 2); +SELECT intDivOrZero(toDecimal128(4.4, 4), toDecimal32(2.2, 2)); +SELECT intDivOrZero(toDecimal256(4.4, 5), 2); +SELECT intDivOrZero(toDecimal256(4.4, 5), toDecimal32(2.2, 2)); +SELECT intDivOrZero(4, toDecimal64(2.2, 2)); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDivOrZero(4, toDecimal128(2.2, 3)); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal128(2.2, 3)); +SELECT intDivOrZero(4, toDecimal256(2.2, 4)); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal256(2.2, 4)); +SELECT intDivOrZero(toDecimal64(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDivOrZero(toDecimal128(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal64(2.2, 2)); +SELECT intDivOrZero(toDecimal64(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDivOrZero(toDecimal128(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal128(2.2, 2)); +SELECT intDivOrZero(toDecimal64(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDivOrZero(toDecimal128(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal256(2.2, 2)); From 1ab29bef622a8de3af7bec194598e3939c9f2d7a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 29 Jan 2024 15:33:09 +0000 Subject: [PATCH 011/105] fixes --- src/Functions/FunctionBinaryArithmetic.h | 54 ++++++++++++++----- src/Functions/IsOperation.h | 6 +-- .../00700_decimal_arithm.reference | 14 ++--- .../01717_int_div_float_too_large_ubsan.sql | 4 +- .../02975_intdiv_with_decimal.reference | 52 +++++++++++------- .../0_stateless/02975_intdiv_with_decimal.sql | 16 ++++++ 6 files changed, 101 insertions(+), 45 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e34514d15fd..831c1cf3aeb 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -153,17 +153,18 @@ public: using ResultDataType = Switch< /// Result must be Integer Case< - only_integer && IsDataTypeDecimal && IsDataTypeDecimal, + only_integer && (IsDataTypeDecimal || IsDataTypeDecimal), Switch< - Case || std::is_same_v, DataTypeInt256>, - Case || std::is_same_v, DataTypeInt128>, - Case || std::is_same_v, DataTypeInt64>, - Case || std::is_same_v, DataTypeInt32>>>, - Case< - only_integer, - Switch< - Case, LeftDataType>, - Case, RightDataType>>>, + Case< + IsDataTypeDecimal || IsDataTypeDecimal, + Switch< + Case, LeftDataType>, + Case, RightDataType>, + Case || std::is_same_v, DataTypeInt256>, + Case || std::is_same_v, DataTypeInt128>, + Case || std::is_same_v, DataTypeInt64>, + Case || std::is_same_v, DataTypeInt32>>>>>, + /// Decimal cases Case || IsDataTypeDecimal), InvalidType>, Case< @@ -1713,12 +1714,37 @@ public: type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } } - else if constexpr ((IsDataTypeDecimal && IsFloatingPoint) || - (IsDataTypeDecimal && IsFloatingPoint)) - type_res = std::make_shared(); + else if constexpr (((IsDataTypeDecimal && IsFloatingPoint) || + (IsDataTypeDecimal && IsFloatingPoint)) && !(is_div_int || is_div_int_or_zero)) + { + if constexpr ((is_div_int || is_div_int_or_zero) && IsDataTypeDecimal) + { + if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v || std::is_same_v) + type_res = std::make_shared(); + else + type_res = std::make_shared(); + } + else if constexpr (is_div_int || is_div_int_or_zero) + { + if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v) + type_res = std::make_shared(); + else if constexpr (std::is_same_v || std::is_same_v) + type_res = std::make_shared(); + else + type_res = std::make_shared(); + } + else + type_res = std::make_shared(); + } else if constexpr (IsDataTypeDecimal) { - if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegral) + if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegralOrExtended) type_res = std::make_shared(); else if constexpr (is_div_int || is_div_int_or_zero) { diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index b36530591ef..b2c7a27d375 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -61,10 +61,8 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; - - static constexpr bool division_allow_decimal = div_floating || modulo; - - static constexpr bool allow_decimal = plus || minus || multiply || division_allow_decimal || least || greatest; + // NOTE: allow_decimal should not fully contain `division` because of divInt + static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; } diff --git a/tests/queries/0_stateless/00700_decimal_arithm.reference b/tests/queries/0_stateless/00700_decimal_arithm.reference index 811946c87e0..20f04696b1b 100644 --- a/tests/queries/0_stateless/00700_decimal_arithm.reference +++ b/tests/queries/0_stateless/00700_decimal_arithm.reference @@ -10,18 +10,18 @@ 63 21 -42 882 -882 2 0 2 0 63 21 -42 882 -882 2 0 2 0 1.00305798474369219219752355409390731264 -0.16305798474369219219752355409390731264 1.490591730234615865843651857942052864 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 0.02 0.005 -63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505 -63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505 -63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505 -63.42 21.42 -41.58 890.82 -890.82 2.02 0.5 2.02 0.5 +63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2 0 +63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2 0 +63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2 0 +63.42 21.42 -41.58 890.82 -890.82 2.02 0.5 2 0 63 -21 42 882 -882 0 2 0 2 63 -21 42 882 -882 0 2 0 2 63 -21 42 882 -882 0 2 0 2 1.00305798474369219219752355409390731264 0.16305798474369219219752355409390731264 -1.490591730234615865843651857942052864 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 -0.00000000000000000000000000000000000001 0.00000000000000000000000000000000000001 -63.42 -21.42 41.58 890.82 -890.82 0.495 1.98 0.495 1.98 +63.42 -21.42 41.58 890.82 -890.82 0.495 1.98 0 2 63.42 -21.42 41.58 890.82 -890.82 -63.42 -21.42 41.58 890.82 -890.82 0.495049504950495049 1.980198019801980198 0.495049504950495049 1.980198019801980198 -63.42 -21.42 41.58 890.82 -890.82 0.49 1.98 0.49 1.98 +63.42 -21.42 41.58 890.82 -890.82 0.495049504950495049 1.980198019801980198 0 2 +63.42 -21.42 41.58 890.82 -890.82 0.49 1.98 0 2 -42 42 42 42 0.42 0.42 0.42 42.42 42.42 42.42 0 0 0 0 0 0 0 0 0 0 42 -42 -42 -42 -0.42 -0.42 -0.42 -42.42 -42.42 -42.42 diff --git a/tests/queries/0_stateless/01717_int_div_float_too_large_ubsan.sql b/tests/queries/0_stateless/01717_int_div_float_too_large_ubsan.sql index c4f26a079f0..dc1e5b37050 100644 --- a/tests/queries/0_stateless/01717_int_div_float_too_large_ubsan.sql +++ b/tests/queries/0_stateless/01717_int_div_float_too_large_ubsan.sql @@ -1,2 +1,2 @@ -SELECT intDiv(9223372036854775807, 0.9998999834060669); -- { serverError 153 } -SELECT intDiv(9223372036854775807, 1.); -- { serverError 153 } +SELECT intDiv(18446744073709551615, 0.9998999834060669); -- { serverError 153 } +SELECT intDiv(18446744073709551615, 1.); -- { serverError 153 } diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.reference b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference index 9c1faab21d7..594dcee975a 100644 --- a/tests/queries/0_stateless/02975_intdiv_with_decimal.reference +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference @@ -24,28 +24,44 @@ 2 2 2 +1 +1 +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 2 2 2 2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 +1 +1 +1 +1 2 2 2 diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.sql b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql index 8fc4b5a9a7d..18e657caa8a 100644 --- a/tests/queries/0_stateless/02975_intdiv_with_decimal.sql +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql @@ -25,6 +25,14 @@ SELECT intDiv(toDecimal256(4.4, 2), toDecimal128(2.2, 2)); SELECT intDiv(toDecimal64(4.4, 2), toDecimal256(2.2, 2)); SELECT intDiv(toDecimal128(4.4, 2), toDecimal256(2.2, 2)); SELECT intDiv(toDecimal256(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDiv(4.2, toDecimal32(2.2, 2)); +SELECT intDiv(4.2, toDecimal64(2.2, 2)); +SELECT intDiv(4.2, toDecimal128(2.2, 2)); +SELECT intDiv(4.2, toDecimal256(2.2, 2)); +SELECT intDiv(toDecimal32(4.4, 2), 2.2); +SELECT intDiv(toDecimal64(4.4, 2), 2.2); +SELECT intDiv(toDecimal128(4.4, 2), 2.2); +SELECT intDiv(toDecimal256(4.4, 2), 2.2); --intDivOrZero-- SELECT intDivOrZero(4,2); SELECT intDivOrZero(toDecimal32(4.4, 2), 2); @@ -52,3 +60,11 @@ SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal128(2.2, 2)); SELECT intDivOrZero(toDecimal64(4.4, 2), toDecimal256(2.2, 2)); SELECT intDivOrZero(toDecimal128(4.4, 2), toDecimal256(2.2, 2)); SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal256(2.2, 2)); +SELECT intDivOrZero(4.2, toDecimal32(2.2, 2)); +SELECT intDivOrZero(4.2, toDecimal64(2.2, 2)); +SELECT intDivOrZero(4.2, toDecimal128(2.2, 2)); +SELECT intDivOrZero(4.2, toDecimal256(2.2, 2)); +SELECT intDivOrZero(toDecimal32(4.4, 2), 2.2); +SELECT intDivOrZero(toDecimal64(4.4, 2), 2.2); +SELECT intDivOrZero(toDecimal128(4.4, 2), 2.2); +SELECT intDivOrZero(toDecimal256(4.4, 2), 2.2); From 0557cdb8a9def2e4c8df81d23cb526153ce023f8 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 30 Jan 2024 15:31:04 +0000 Subject: [PATCH 012/105] fix due to review --- src/Functions/FunctionBinaryArithmetic.h | 40 ++++++++++-------------- 1 file changed, 17 insertions(+), 23 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 831c1cf3aeb..62a50f5e0c2 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -156,14 +156,18 @@ public: only_integer && (IsDataTypeDecimal || IsDataTypeDecimal), Switch< Case< - IsDataTypeDecimal || IsDataTypeDecimal, + IsDataTypeDecimal, + Switch< + Case, DataTypeInt256>, + Case, DataTypeInt128>, + Case, DataTypeInt64>, + Case, DataTypeInt32>>>, + Case< + IsDataTypeDecimal, Switch< Case, LeftDataType>, - Case, RightDataType>, - Case || std::is_same_v, DataTypeInt256>, - Case || std::is_same_v, DataTypeInt128>, - Case || std::is_same_v, DataTypeInt64>, - Case || std::is_same_v, DataTypeInt32>>>>>, + Case, DataTypeInt64>, + Case, DataTypeInt32>>>>>, /// Decimal cases Case || IsDataTypeDecimal), InvalidType>, @@ -1684,11 +1688,11 @@ public: { if constexpr (is_div_int || is_div_int_or_zero) { - if constexpr (std::is_same_v || std::is_same_v) + if constexpr (std::is_same_v) type_res = std::make_shared(); - else if constexpr (std::is_same_v || std::is_same_v) + else if constexpr (std::is_same_v) type_res = std::make_shared(); - else if constexpr (std::is_same_v || std::is_same_v) + else if constexpr (std::is_same_v) type_res = std::make_shared(); else type_res = std::make_shared(); @@ -1723,18 +1727,14 @@ public: type_res = std::make_shared(); else if constexpr (std::is_same_v) type_res = std::make_shared(); - else if constexpr (std::is_same_v || std::is_same_v) + else if constexpr (std::is_same_v) type_res = std::make_shared(); else type_res = std::make_shared(); } else if constexpr (is_div_int || is_div_int_or_zero) { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v || std::is_same_v) + if constexpr (std::is_same_v) type_res = std::make_shared(); else type_res = std::make_shared(); @@ -1744,9 +1744,7 @@ public: } else if constexpr (IsDataTypeDecimal) { - if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegralOrExtended) - type_res = std::make_shared(); - else if constexpr (is_div_int || is_div_int_or_zero) + if constexpr (is_div_int || is_div_int_or_zero) { if constexpr (std::is_same_v) type_res = std::make_shared(); @@ -1766,11 +1764,7 @@ public: type_res = std::make_shared(); else if constexpr (is_div_int || is_div_int_or_zero) { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) + if constexpr (std::is_same_v) type_res = std::make_shared(); else type_res = std::make_shared(); From 998c56fc3d3602a1151c7e310863e12666e595e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Jan 2024 17:36:34 +0100 Subject: [PATCH 013/105] Move code --- src/Compression/CompressionCodecT64.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 42c6a18aa77..3ddc56fe4f6 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -168,6 +168,7 @@ TypeIndex baseType(TypeIndex type_idx) return TypeIndex::Int16; case TypeIndex::Int32: case TypeIndex::Decimal32: + case TypeIndex::Date32: return TypeIndex::Int32; case TypeIndex::Int64: case TypeIndex::Decimal64: @@ -180,8 +181,6 @@ TypeIndex baseType(TypeIndex type_idx) case TypeIndex::Enum16: case TypeIndex::Date: return TypeIndex::UInt16; - case TypeIndex::Date32: - return TypeIndex::Int32; case TypeIndex::UInt32: case TypeIndex::DateTime: case TypeIndex::IPv4: From 0576aa2b7fd060c68f482f8205575bd904356ebe Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 30 Jan 2024 16:45:36 +0000 Subject: [PATCH 014/105] fix fuzzer --- src/Functions/FunctionBinaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 62a50f5e0c2..e31183573c3 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1719,7 +1719,7 @@ public: } } else if constexpr (((IsDataTypeDecimal && IsFloatingPoint) || - (IsDataTypeDecimal && IsFloatingPoint)) && !(is_div_int || is_div_int_or_zero)) + (IsDataTypeDecimal && IsFloatingPoint))) { if constexpr ((is_div_int || is_div_int_or_zero) && IsDataTypeDecimal) { @@ -1760,7 +1760,7 @@ public: } else if constexpr (IsDataTypeDecimal) { - if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegral) + if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegralOrExtended) type_res = std::make_shared(); else if constexpr (is_div_int || is_div_int_or_zero) { From d5eec2d85b616a13fe5123ab4cdc7f0d3471e425 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 1 Feb 2024 16:27:57 +0000 Subject: [PATCH 015/105] trying to fix (casting the result to int) --- base/base/Decimal_fwd.h | 6 + src/Functions/FunctionBinaryArithmetic.h | 137 ++++++++---------- .../02975_intdiv_with_decimal.reference | 20 +-- .../0_stateless/02975_intdiv_with_decimal.sql | 8 +- 4 files changed, 84 insertions(+), 87 deletions(-) diff --git a/base/base/Decimal_fwd.h b/base/base/Decimal_fwd.h index 589d6224917..beb228cea3c 100644 --- a/base/base/Decimal_fwd.h +++ b/base/base/Decimal_fwd.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace wide { @@ -44,3 +45,8 @@ concept is_over_big_int = || std::is_same_v || std::is_same_v; } + +template <> struct is_signed { static constexpr bool value = true; }; +template <> struct is_signed { static constexpr bool value = true; }; +template <> struct is_signed { static constexpr bool value = true; }; +template <> struct is_signed { static constexpr bool value = true; }; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e31183573c3..9b0afee5053 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -152,22 +152,7 @@ public: /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). using ResultDataType = Switch< /// Result must be Integer - Case< - only_integer && (IsDataTypeDecimal || IsDataTypeDecimal), - Switch< - Case< - IsDataTypeDecimal, - Switch< - Case, DataTypeInt256>, - Case, DataTypeInt128>, - Case, DataTypeInt64>, - Case, DataTypeInt32>>>, - Case< - IsDataTypeDecimal, - Switch< - Case, LeftDataType>, - Case, DataTypeInt64>, - Case, DataTypeInt32>>>>>, + Case::div_int || IsOperation::div_int_or_zero, DataTypeFromFieldType>, /// Decimal cases Case || IsDataTypeDecimal), InvalidType>, @@ -1687,16 +1672,7 @@ public: if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { if constexpr (is_div_int || is_div_int_or_zero) - { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else - type_res = std::make_shared(); - } + type_res = std::make_shared(); else { if constexpr (is_division) @@ -1721,54 +1697,22 @@ public: else if constexpr (((IsDataTypeDecimal && IsFloatingPoint) || (IsDataTypeDecimal && IsFloatingPoint))) { - if constexpr ((is_div_int || is_div_int_or_zero) && IsDataTypeDecimal) - { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else - type_res = std::make_shared(); - } - else if constexpr (is_div_int || is_div_int_or_zero) - { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else - type_res = std::make_shared(); - } + if constexpr (is_div_int || is_div_int_or_zero) + type_res = std::make_shared(); else type_res = std::make_shared(); } else if constexpr (IsDataTypeDecimal) { if constexpr (is_div_int || is_div_int_or_zero) - { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else if constexpr (std::is_same_v) - type_res = std::make_shared(); - else - type_res = std::make_shared(); - } + type_res = std::make_shared(); else type_res = std::make_shared(left.getPrecision(), left.getScale()); } else if constexpr (IsDataTypeDecimal) { - if constexpr ((is_div_int || is_div_int_or_zero) && IsIntegralOrExtended) - type_res = std::make_shared(); - else if constexpr (is_div_int || is_div_int_or_zero) - { - if constexpr (std::is_same_v) - type_res = std::make_shared(); - else - type_res = std::make_shared(); - } + if constexpr (is_div_int || is_div_int_or_zero) + type_res = std::make_shared(); else type_res = std::make_shared(right.getPrecision(), right.getScale()); } @@ -2089,10 +2033,8 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A constexpr bool decimal_with_float = (IsDataTypeDecimal && IsFloatingPoint) || (IsFloatingPoint && IsDataTypeDecimal); - constexpr bool is_div_int_with_decimal = (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal); - - using T0 = std::conditional_t>; - using T1 = std::conditional_t>; + using T0 = std::conditional_t; + using T1 = std::conditional_t; using ResultType = typename ResultDataType::FieldType; using ColVecT0 = ColumnVectorOrDecimal; using ColVecT1 = ColumnVectorOrDecimal; @@ -2108,12 +2050,6 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A left_col = castColumn(arguments[0], converted_type); right_col = castColumn(arguments[1], converted_type); } - else if constexpr (is_div_int_with_decimal) - { - const auto converted_type = std::make_shared(); - left_col = castColumn(arguments[0], converted_type); - right_col = castColumn(arguments[1], converted_type); - } else { left_col = arguments[0].column; @@ -2139,6 +2075,61 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A col_left_size, right_nullmap); } + else if constexpr (!decimal_with_float && (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal)) + { + using DecimalResultType = Switch< + Case< + IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, + LeftDataType>, + Case && IsDataTypeDecimal, RightDataType>, + Case && IsIntegralOrExtended, LeftDataType>, + Case && IsIntegralOrExtended, RightDataType>, + + /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) + Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, + Case && !IsIntegralOrExtendedOrDecimal, InvalidType>>; /// Determine result decimal type as it would be with usual division (as we determine BinaryOperationTraits::ResultType) + + if constexpr (!std::is_same_v) + { + DataTypePtr type_res; + if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) + { + if constexpr (is_division) + { + if (context->getSettingsRef().decimal_check_overflow) + { + /// Check overflow by using operands scale (based on big decimal division implementation details): + /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers + /// i.e. int_operand = decimal_operand*10^scale + /// For division, left operand will be scaled by right operand scale also to do big integer division, + /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale + /// So, we can check upfront possible overflow just by checking max scale used for left operand + /// Note: it doesn't detect all possible overflow during big decimal division + if (left.getScale() + right.getScale() > DecimalResultType::maxPrecision()) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); + } + } + DecimalResultType result_type = decimalResultType(left, right); + type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); + } + else if constexpr (IsDataTypeDecimal) + type_res = std::make_shared(left.getPrecision(), left.getScale()); + else + type_res = std::make_shared(right.getPrecision(), right.getScale()); + // Create result decimal type somehow, maybe similar to how we do it in getReturnTypeImplStatic + + auto res = executeNumericWithDecimal( + left, right, + col_left_const, col_right_const, + col_left, col_right, + col_left_size, + right_nullmap); + + auto col = ColumnWithTypeAndName(res, type_res, name); + return castColumn(col, std::make_shared()); + } + return nullptr; + } else // can't avoid else and another indentation level, otherwise the compiler would try to instantiate // ColVecResult for Decimals which would lead to a compile error. { diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.reference b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference index 594dcee975a..5540734ae4c 100644 --- a/tests/queries/0_stateless/02975_intdiv_with_decimal.reference +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.reference @@ -1,19 +1,19 @@ 2 2 +1 +2 +2 2 2 2 2 2 2 +1 2 +1 2 -2 -2 -2 -2 -2 -2 +1 2 2 2 @@ -34,6 +34,7 @@ 2 2 2 +1 2 2 2 @@ -42,12 +43,11 @@ 2 2 2 +1 2 +1 2 -2 -2 -2 -2 +1 2 2 2 diff --git a/tests/queries/0_stateless/02975_intdiv_with_decimal.sql b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql index 18e657caa8a..0911a481251 100644 --- a/tests/queries/0_stateless/02975_intdiv_with_decimal.sql +++ b/tests/queries/0_stateless/02975_intdiv_with_decimal.sql @@ -13,9 +13,9 @@ SELECT intDiv(toDecimal256(4.4, 5), toDecimal32(2.2, 2)); SELECT intDiv(4, toDecimal64(2.2, 2)); SELECT intDiv(toDecimal32(4.4, 2), toDecimal64(2.2, 2)); SELECT intDiv(4, toDecimal128(2.2, 3)); -SELECT intDiv(toDecimal32(4.4, 2), toDecimal128(2.2, 3)); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal128(2.2, 2)); SELECT intDiv(4, toDecimal256(2.2, 4)); -SELECT intDiv(toDecimal32(4.4, 2), toDecimal256(2.2, 4)); +SELECT intDiv(toDecimal32(4.4, 2), toDecimal256(2.2, 2)); SELECT intDiv(toDecimal64(4.4, 2), toDecimal64(2.2, 2)); SELECT intDiv(toDecimal128(4.4, 2), toDecimal64(2.2, 2)); SELECT intDiv(toDecimal256(4.4, 2), toDecimal64(2.2, 2)); @@ -48,9 +48,9 @@ SELECT intDivOrZero(toDecimal256(4.4, 5), toDecimal32(2.2, 2)); SELECT intDivOrZero(4, toDecimal64(2.2, 2)); SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal64(2.2, 2)); SELECT intDivOrZero(4, toDecimal128(2.2, 3)); -SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal128(2.2, 3)); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal128(2.2, 2)); SELECT intDivOrZero(4, toDecimal256(2.2, 4)); -SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal256(2.2, 4)); +SELECT intDivOrZero(toDecimal32(4.4, 2), toDecimal256(2.2, 2)); SELECT intDivOrZero(toDecimal64(4.4, 2), toDecimal64(2.2, 2)); SELECT intDivOrZero(toDecimal128(4.4, 2), toDecimal64(2.2, 2)); SELECT intDivOrZero(toDecimal256(4.4, 2), toDecimal64(2.2, 2)); From ff21aa9a19a9a2ebd9e16aa32ea1a10d4e988abe Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 1 Feb 2024 19:47:05 +0000 Subject: [PATCH 016/105] Don't infer floats in exponential notation by default --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/SchemaInferenceUtils.cpp | 16 +++++-- src/IO/readFloatText.cpp | 3 ++ src/IO/readFloatText.h | 48 +++++++++++-------- ...02982_dont_infer_exponent_floats.reference | 2 + .../02982_dont_infer_exponent_floats.sql | 3 ++ 9 files changed, 51 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference create mode 100644 tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4bb48cb3a29..a892c3bb58e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1008,6 +1008,7 @@ class IColumn; M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ + M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats", 0) \ M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ad04ee79995..a70daf8e1c7 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -84,6 +84,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.2", {{"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}}}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 62cbadec4f4..78378168d02 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -226,6 +226,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.try_infer_integers = settings.input_format_try_infer_integers; format_settings.try_infer_dates = settings.input_format_try_infer_dates; format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes; + format_settings.try_infer_exponent_floats = settings.input_format_try_infer_exponent_floats; format_settings.markdown.escape_special_characters = settings.output_format_markdown_escape_special_characters; format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 30e4dd04513..ba7cd6055a7 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -46,6 +46,7 @@ struct FormatSettings bool try_infer_integers = false; bool try_infer_dates = false; bool try_infer_datetimes = false; + bool try_infer_exponent_floats = false; enum class DateTimeInputFormat { diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 2cfcff75edd..06b52e7a7a2 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -865,6 +866,13 @@ namespace return std::make_shared(nested_types); } + bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings) + { + if (settings.try_infer_exponent_floats) + return tryReadFloatText(value, buf); + return tryReadFloatTextNoExponent(value, buf); + } + DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings) { if (buf.eof()) @@ -903,7 +911,7 @@ namespace buf.position() = number_start; } - if (tryReadFloatText(tmp_float, buf)) + if (tryReadFloat(tmp_float, buf, settings)) { if (read_int && buf.position() == int_end) return std::make_shared(); @@ -937,7 +945,7 @@ namespace peekable_buf.rollbackToCheckpoint(true); } - if (tryReadFloatText(tmp_float, peekable_buf)) + if (tryReadFloat(tmp_float, peekable_buf, settings)) { /// Float parsing reads no fewer bytes than integer parsing, /// so position of the buffer is either the same, or further. @@ -949,7 +957,7 @@ namespace return std::make_shared(); } } - else if (tryReadFloatText(tmp_float, buf)) + else if (tryReadFloat(tmp_float, buf, settings)) { return std::make_shared(); } @@ -1390,7 +1398,7 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting buf.position() = buf.buffer().begin(); Float64 tmp; - if (tryReadFloatText(tmp, buf) && buf.eof()) + if (tryReadFloat(tmp, buf, settings) && buf.eof()) return std::make_shared(); return nullptr; diff --git a/src/IO/readFloatText.cpp b/src/IO/readFloatText.cpp index d1143f7c62c..17ccc1b25b7 100644 --- a/src/IO/readFloatText.cpp +++ b/src/IO/readFloatText.cpp @@ -67,4 +67,7 @@ template void readFloatText(Float64 &, ReadBuffer &); template bool tryReadFloatText(Float32 &, ReadBuffer &); template bool tryReadFloatText(Float64 &, ReadBuffer &); +template bool tryReadFloatTextNoExponent(Float32 &, ReadBuffer &); +template bool tryReadFloatTextNoExponent(Float64 &, ReadBuffer &); + } diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index 23e904f305a..51964636389 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -324,7 +324,7 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf) } -template +template ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) { static_assert(std::is_same_v || std::is_same_v, "Argument for readFloatTextImpl must be float or double"); @@ -395,30 +395,33 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) after_point_exponent = (read_digits > significant_digits ? -significant_digits : static_cast(-read_digits)) - after_point_num_leading_zeros; } - if (checkChar('e', in) || checkChar('E', in)) + if constexpr (allow_exponent) { - if (in.eof()) + if (checkChar('e', in) || checkChar('E', in)) { - if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent"); - else - return false; - } + if (in.eof()) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent"); + else + return false; + } - bool exponent_negative = false; - if (*in.position() == '-') - { - exponent_negative = true; - ++in.position(); - } - else if (*in.position() == '+') - { - ++in.position(); - } + bool exponent_negative = false; + if (*in.position() == '-') + { + exponent_negative = true; + ++in.position(); + } + else if (*in.position() == '+') + { + ++in.position(); + } - readUIntTextUpToNSignificantDigits<4>(exponent, in); - if (exponent_negative) - exponent = -exponent; + readUIntTextUpToNSignificantDigits<4>(exponent, in); + if (exponent_negative) + exponent = -exponent; + } } if (after_point) @@ -604,4 +607,7 @@ template bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { retu template void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); } template bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); } +/// Don't read exponent part of the number. +template bool tryReadFloatTextNoExponent(T & x, ReadBuffer & in) { return readFloatTextFastImpl(x, in); } + } diff --git a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference new file mode 100644 index 00000000000..b6d1ff865e5 --- /dev/null +++ b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference @@ -0,0 +1,2 @@ +c1 Nullable(String) +c1 Nullable(Float64) diff --git a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql new file mode 100644 index 00000000000..17f62557fc2 --- /dev/null +++ b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql @@ -0,0 +1,3 @@ +DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 0; +DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 1; + From e79ddd54afa54dc0c964774899f7250514741004 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Sat, 3 Feb 2024 00:56:37 +0000 Subject: [PATCH 017/105] fix tests --- src/Functions/FunctionBinaryArithmetic.h | 19 +++---------------- .../00700_decimal_arithm.reference | 6 +++--- 2 files changed, 6 insertions(+), 19 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 9b0afee5053..4d768311aaf 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -146,7 +146,6 @@ private: /// it's not correct for Decimal public: static constexpr bool allow_decimal = IsOperation::allow_decimal; - static constexpr bool only_integer = IsOperation::div_int || IsOperation::div_int_or_zero; /// Appropriate result type for binary operator on numeric types. "Date" can also mean /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). @@ -626,7 +625,10 @@ private: if constexpr (op_case == OpCase::RightConstant) { if ((*right_nullmap)[0]) + { + c[0] = ResultType(); return; + } for (size_t i = 0; i < size; ++i) c[i] = apply_func(undec(a[i]), undec(b)); @@ -2094,21 +2096,6 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A DataTypePtr type_res; if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { - if constexpr (is_division) - { - if (context->getSettingsRef().decimal_check_overflow) - { - /// Check overflow by using operands scale (based on big decimal division implementation details): - /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers - /// i.e. int_operand = decimal_operand*10^scale - /// For division, left operand will be scaled by right operand scale also to do big integer division, - /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale - /// So, we can check upfront possible overflow just by checking max scale used for left operand - /// Note: it doesn't detect all possible overflow during big decimal division - if (left.getScale() + right.getScale() > DecimalResultType::maxPrecision()) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); - } - } DecimalResultType result_type = decimalResultType(left, right); type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } diff --git a/tests/queries/0_stateless/00700_decimal_arithm.reference b/tests/queries/0_stateless/00700_decimal_arithm.reference index 20f04696b1b..109c0632fb1 100644 --- a/tests/queries/0_stateless/00700_decimal_arithm.reference +++ b/tests/queries/0_stateless/00700_decimal_arithm.reference @@ -18,10 +18,10 @@ 63 -21 42 882 -882 0 2 0 2 63 -21 42 882 -882 0 2 0 2 1.00305798474369219219752355409390731264 0.16305798474369219219752355409390731264 -1.490591730234615865843651857942052864 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 -0.00000000000000000000000000000000000001 0.00000000000000000000000000000000000001 -63.42 -21.42 41.58 890.82 -890.82 0.495 1.98 0 2 +63.42 -21.42 41.58 890.82 -890.82 0.495 1.98 0 1 63.42 -21.42 41.58 890.82 -890.82 -63.42 -21.42 41.58 890.82 -890.82 0.495049504950495049 1.980198019801980198 0 2 -63.42 -21.42 41.58 890.82 -890.82 0.49 1.98 0 2 +63.42 -21.42 41.58 890.82 -890.82 0.495049504950495049 1.980198019801980198 0 1 +63.42 -21.42 41.58 890.82 -890.82 0.49 1.98 0 1 -42 42 42 42 0.42 0.42 0.42 42.42 42.42 42.42 0 0 0 0 0 0 0 0 0 0 42 -42 -42 -42 -0.42 -0.42 -0.42 -42.42 -42.42 -42.42 From 88b6b38a748e994695b078fa03bd6dd5b9210ee0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 04:21:57 +0100 Subject: [PATCH 018/105] Check stack size in Parser --- src/Parsers/IParser.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index d53b58baa7c..6be33e43920 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -73,6 +74,16 @@ public: if (unlikely(max_depth > 0 && depth > max_depth)) throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "Maximum parse depth ({}) exceeded. " "Consider rising max_parser_depth parameter.", max_depth); + + /** Sometimes the maximum parser depth can be set to a high value by the user, + * but we still want to avoid stack overflow. + * For this purpose, we can use the checkStackSize function, but it is too heavy. + * The solution is to check not too frequently. + * The frequency 128 is arbitrary, but not too large, not too small, + * and a power of two to simplify the division. + */ + if (depth % 8192 == 0) + checkStackSize(); } ALWAYS_INLINE void decreaseDepth() From 5b791ddec894e1587db7a726e68890b479312145 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 04:23:43 +0100 Subject: [PATCH 019/105] Check stack size in Parser --- src/Parsers/IParser.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 6be33e43920..99dbc39f06f 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -79,7 +79,7 @@ public: * but we still want to avoid stack overflow. * For this purpose, we can use the checkStackSize function, but it is too heavy. * The solution is to check not too frequently. - * The frequency 128 is arbitrary, but not too large, not too small, + * The frequency is arbitrary, but not too large, not too small, * and a power of two to simplify the division. */ if (depth % 8192 == 0) From a652bd98b25a9cc04a01702815ec151b0f402b45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 04:30:48 +0100 Subject: [PATCH 020/105] Add a test --- .../0_stateless/02985_parser_check_stack_size.reference | 1 + tests/queries/0_stateless/02985_parser_check_stack_size.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02985_parser_check_stack_size.reference create mode 100755 tests/queries/0_stateless/02985_parser_check_stack_size.sh diff --git a/tests/queries/0_stateless/02985_parser_check_stack_size.reference b/tests/queries/0_stateless/02985_parser_check_stack_size.reference new file mode 100644 index 00000000000..f83e0818db2 --- /dev/null +++ b/tests/queries/0_stateless/02985_parser_check_stack_size.reference @@ -0,0 +1 @@ +TOO_DEEP diff --git a/tests/queries/0_stateless/02985_parser_check_stack_size.sh b/tests/queries/0_stateless/02985_parser_check_stack_size.sh new file mode 100755 index 00000000000..c91a0a3eacc --- /dev/null +++ b/tests/queries/0_stateless/02985_parser_check_stack_size.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "select 'create table test (x ' || repeat('Array(', 10000) || 'UInt64' || repeat(')', 10000) || ') engine=Memory' format TSVRaw" | $CLICKHOUSE_CURL "${CLICKHOUSE_URL}&max_parser_depth=100000" --data-binary @- | grep -o -F 'TOO_DEEP' From 7016e1d397de7f50dc39b59f370924fc93f1a889 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 15:43:58 +0100 Subject: [PATCH 021/105] Maybe better --- src/Parsers/IParser.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 99dbc39f06f..d5c9d48d730 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -82,7 +82,12 @@ public: * The frequency is arbitrary, but not too large, not too small, * and a power of two to simplify the division. */ - if (depth % 8192 == 0) +#ifdef USE_MUSL + static constexpr uint32_t check_frequency = 128; +#else + static constexpr uint32_t check_frequency = 8192; +#endif + if (depth % check_frequency == 0) checkStackSize(); } From aa004cabffdc5a7a40b0090f3540da0464c0924b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 02:00:05 +0100 Subject: [PATCH 022/105] Fix tests --- src/Parsers/IParser.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index d5c9d48d730..e8f95954e70 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -82,7 +82,7 @@ public: * The frequency is arbitrary, but not too large, not too small, * and a power of two to simplify the division. */ -#ifdef USE_MUSL +#if defined(USE_MUSL) || defined(SANITIZER) static constexpr uint32_t check_frequency = 128; #else static constexpr uint32_t check_frequency = 8192; From c7c05c9881362079fa6a02430fc5cd63dfa65bb7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 16:22:10 +0100 Subject: [PATCH 023/105] Do not pull mutations if pulling replication log had been stopped Right now, mutations can be pulled even after: SYSTEM STOP PULLING REPLICATION LOG Since they pulled from two places: - StorageReplicatedMergeTree::mutationsUpdatingTask() - ReplicatedMergeTreeQueue::pullLogsToQueue() And only the last one checks action blocker. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 8d921bdcb1c..e26a36202dd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -860,6 +860,9 @@ ActiveDataPartSet getPartNamesToMutate( int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback) { + if (pull_log_blocker.isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Log pulling is cancelled"); + std::lock_guard lock(update_mutations_mutex); Coordination::Stat mutations_stat; From eb44faf867152713ebbbe3753a821acf803f8e37 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 12 Feb 2024 20:41:46 +0000 Subject: [PATCH 024/105] Test to reproduce absence of closing record in query_log --- src/Common/FailPoint.cpp | 5 ++-- src/Common/FailPoint.h | 2 -- src/Interpreters/executeQuery.cpp | 18 ++++++++---- .../__init__.py | 0 .../test_insert_exception_over_http/test.py | 29 +++++++++++++++++++ 5 files changed, 45 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_insert_exception_over_http/__init__.py create mode 100644 tests/integration/test_insert_exception_over_http/test.py diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index f29aee0cdcc..e0639df29b4 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -1,3 +1,4 @@ +#include "config.h" #include #include #include @@ -6,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -44,7 +44,8 @@ static struct InitFiu REGULAR(dummy_failpoint) \ REGULAR(prefetched_reader_pool_failpoint) \ PAUSEABLE_ONCE(dummy_pausable_failpoint_once) \ - PAUSEABLE(dummy_pausable_failpoint) + PAUSEABLE(dummy_pausable_failpoint) \ + ONCE(execute_query_calling_empty_set_result_func_on_exception) namespace FailPoints { diff --git a/src/Common/FailPoint.h b/src/Common/FailPoint.h index b9eb13903a6..a0081d67081 100644 --- a/src/Common/FailPoint.h +++ b/src/Common/FailPoint.h @@ -1,5 +1,4 @@ #pragma once -#include "config.h" #include #include @@ -18,7 +17,6 @@ #pragma clang diagnostic pop #endif -#include #include namespace DB diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8b36790a269..d724d350958 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -105,6 +106,10 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } +namespace FailPoints +{ + extern const char execute_query_calling_empty_set_result_func_on_exception[]; +} static void checkASTSizeLimits(const IAST & ast, const Settings & settings) { @@ -1361,7 +1366,7 @@ void executeQuery( BlockIO streams; OutputFormatPtr output_format; - auto update_format_for_exception_if_needed = [&]() + auto update_format_on_exception_if_needed = [&]() { if (!output_format) { @@ -1374,10 +1379,13 @@ void executeQuery( /// Force an update of the headers before we start writing result_details.content_type = output_format->getContentType(); result_details.format = format_name; + if (set_result_details) { - set_result_details(result_details); + /// reset set_result_details func to avoid calling in SCOPE_EXIT() + auto set_result_details_copy = set_result_details; set_result_details = nullptr; + set_result_details_copy(result_details); } } } @@ -1397,7 +1405,7 @@ void executeQuery( { if (handle_exception_in_output_format) { - update_format_for_exception_if_needed(); + update_format_on_exception_if_needed(); if (output_format) handle_exception_in_output_format(*output_format); } @@ -1427,7 +1435,7 @@ void executeQuery( const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); std::string compression_method; - if (ast_query_with_output->compression) + if (ast_query_with_output->compressiong { const auto & compression_method_node = ast_query_with_output->compression->as(); compression_method = compression_method_node.value.safeGet(); @@ -1500,7 +1508,7 @@ void executeQuery( { if (handle_exception_in_output_format) { - update_format_for_exception_if_needed(); + update_format_on_exception_if_needed(); if (output_format) handle_exception_in_output_format(*output_format); } diff --git a/tests/integration/test_insert_exception_over_http/__init__.py b/tests/integration/test_insert_exception_over_http/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_insert_exception_over_http/test.py b/tests/integration/test_insert_exception_over_http/test.py new file mode 100644 index 00000000000..3630b8379aa --- /dev/null +++ b/tests/integration/test_insert_exception_over_http/test.py @@ -0,0 +1,29 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance("instance", with_zookeeper=True) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_insert_exception_over_http(start_cluster): + instance.query("DROP TABLE IF EXISTS tt SYNC"); + instance.query("CREATE TABLE tt (KeyID UInt32) Engine = ReplicatedMergeTree('/test_insert_exception_over_http/tt', 'r1') ORDER BY (KeyID)"); + instance.query("SYSTEM ENABLE FAILPOINT execute_query_calling_empty_set_result_func_on_exception"); + + assert True == instance.http_query_and_get_error("insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)").startswith("500 Internal Server Error") + + assert "0\n" == instance.query("select count() from tt") + instance.query("SYSTEM FLUSH LOGS") + assert "2\n" == instance.query("select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday()") + + instance.query("DROP TABLE tt SYNC") From f5880d7201e0c939629d8115cf2cbfb148d0966c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Feb 2024 20:55:20 +0000 Subject: [PATCH 025/105] Automatic style fix --- .../test_insert_exception_over_http/test.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_insert_exception_over_http/test.py b/tests/integration/test_insert_exception_over_http/test.py index 3630b8379aa..962c20d6bff 100644 --- a/tests/integration/test_insert_exception_over_http/test.py +++ b/tests/integration/test_insert_exception_over_http/test.py @@ -16,14 +16,22 @@ def start_cluster(): def test_insert_exception_over_http(start_cluster): - instance.query("DROP TABLE IF EXISTS tt SYNC"); - instance.query("CREATE TABLE tt (KeyID UInt32) Engine = ReplicatedMergeTree('/test_insert_exception_over_http/tt', 'r1') ORDER BY (KeyID)"); - instance.query("SYSTEM ENABLE FAILPOINT execute_query_calling_empty_set_result_func_on_exception"); + instance.query("DROP TABLE IF EXISTS tt SYNC") + instance.query( + "CREATE TABLE tt (KeyID UInt32) Engine = ReplicatedMergeTree('/test_insert_exception_over_http/tt', 'r1') ORDER BY (KeyID)" + ) + instance.query( + "SYSTEM ENABLE FAILPOINT execute_query_calling_empty_set_result_func_on_exception" + ) - assert True == instance.http_query_and_get_error("insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)").startswith("500 Internal Server Error") + assert True == instance.http_query_and_get_error( + "insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)" + ).startswith("500 Internal Server Error") assert "0\n" == instance.query("select count() from tt") instance.query("SYSTEM FLUSH LOGS") - assert "2\n" == instance.query("select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday()") + assert "2\n" == instance.query( + "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday()" + ) instance.query("DROP TABLE tt SYNC") From d1167d0b04f3bec6abb6155c0e1f40851fa5d5e9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 12 Feb 2024 20:55:19 +0000 Subject: [PATCH 026/105] Fix: corrupted source + fail point --- src/Interpreters/executeQuery.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index d724d350958..345472fe4d0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1380,6 +1380,11 @@ void executeQuery( result_details.content_type = output_format->getContentType(); result_details.format = format_name; + fiu_do_on(FailPoint::execute_query_calling_empty_set_result_func_on_exception, { + set_result_details = nullptr; + set_result_details(result_details); + }); + if (set_result_details) { /// reset set_result_details func to avoid calling in SCOPE_EXIT() @@ -1435,7 +1440,7 @@ void executeQuery( const auto & out_file = typeid_cast(*ast_query_with_output->out_file).value.safeGet(); std::string compression_method; - if (ast_query_with_output->compressiong + if (ast_query_with_output->compression) { const auto & compression_method_node = ast_query_with_output->compression->as(); compression_method = compression_method_node.value.safeGet(); From 58e5d7876b40879dedecc34d0bcef3f435508b1a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 20:32:23 +0100 Subject: [PATCH 027/105] Allow uuid in replica_path if CREATE TABLE explicitly has it Right now the query with {uuid} macro without ON CLUSTER fails: CREATE TABLE x UUID 'aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa' (key Int) ENGINE = ReplicatedMergeTree('/tables/{database}/{uuid}', 'r1') ORDER BY tuple(); There is a workaround right now to use ATTACH instead of CREATE, but ATTACH is not CREATE. CREATE still useful for proper RESTORE without ON CLUSTER. So this patch allows this syntax, but only if UUID had been explicitly passed in the query. This looks safe. Signed-off-by: Azat Khuzhin --- src/Parsers/ASTCreateQuery.h | 1 + src/Parsers/ParserCreateQuery.cpp | 2 ++ .../MergeTree/registerStorageMergeTree.cpp | 3 ++- .../02990_rmt_replica_path_uuid.reference | 4 ++++ .../02990_rmt_replica_path_uuid.sql | 23 +++++++++++++++++++ 5 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02990_rmt_replica_path_uuid.reference create mode 100644 tests/queries/0_stateless/02990_rmt_replica_path_uuid.sql diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 49a0140625c..b1209e72b61 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -96,6 +96,7 @@ public: bool is_populate{false}; bool is_create_empty{false}; /// CREATE TABLE ... EMPTY AS SELECT ... bool replace_view{false}; /// CREATE OR REPLACE VIEW + bool has_uuid{false}; // CREATE TABLE x UUID '...' ASTColumns * columns_list = nullptr; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 1f6f68c9d8e..27c6e6258e3 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -684,6 +684,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->database = table_id->getDatabase(); query->table = table_id->getTable(); query->uuid = table_id->uuid; + query->has_uuid = table_id->uuid != UUIDHelpers::Nil; if (query->database) query->children.push_back(query->database); @@ -783,6 +784,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->database = table_id->getDatabase(); query->table = table_id->getTable(); query->uuid = table_id->uuid; + query->has_uuid = table_id->uuid != UUIDHelpers::Nil; query->cluster = cluster_str; if (query->database) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 8e646e48f16..ca4f19a24e8 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -315,7 +315,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries - bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; + /// and if UUID was explicitly passed in CREATE TABLE (like for ATTACH) + bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach || args.query.has_uuid; auto expand_macro = [&] (ASTLiteral * ast_zk_path, ASTLiteral * ast_replica_name) { diff --git a/tests/queries/0_stateless/02990_rmt_replica_path_uuid.reference b/tests/queries/0_stateless/02990_rmt_replica_path_uuid.reference new file mode 100644 index 00000000000..5521c015fcf --- /dev/null +++ b/tests/queries/0_stateless/02990_rmt_replica_path_uuid.reference @@ -0,0 +1,4 @@ +aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa +/tables/default/aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa/replicas/r1 +aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa +/tables/default/aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa/replicas/r1 diff --git a/tests/queries/0_stateless/02990_rmt_replica_path_uuid.sql b/tests/queries/0_stateless/02990_rmt_replica_path_uuid.sql new file mode 100644 index 00000000000..4fcdff2910f --- /dev/null +++ b/tests/queries/0_stateless/02990_rmt_replica_path_uuid.sql @@ -0,0 +1,23 @@ +-- Tags: no-parallel, no-ordinary-database, no-replicated-database +-- Tag no-parallel: static UUID +-- Tag no-ordinary-database: requires UUID +-- Tag no-replicated-database: executes with ON CLUSTER anyway + +-- Ignore "ATTACH TABLE query with full table definition is not recommended" +-- Ignore BAD_ARGUMENTS +SET send_logs_level='fatal'; + +DROP TABLE IF EXISTS x; + +ATTACH TABLE x UUID 'aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa' (key Int) ENGINE = ReplicatedMergeTree('/tables/{database}/{uuid}', 'r1') ORDER BY tuple(); +SELECT uuid FROM system.tables WHERE database = currentDatabase() and table = 'x'; +SELECT replica_path FROM system.replicas WHERE database = currentDatabase() and table = 'x'; +DROP TABLE x; + +-- {uuid} macro forbidden for CREATE TABLE without explicit UUID +CREATE TABLE x (key Int) ENGINE = ReplicatedMergeTree('/tables/{database}/{uuid}', 'r1') ORDER BY tuple(); -- { serverError BAD_ARGUMENTS } + +CREATE TABLE x UUID 'aaaaaaaa-1111-2222-3333-aaaaaaaaaaaa' (key Int) ENGINE = ReplicatedMergeTree('/tables/{database}/{uuid}', 'r1') ORDER BY tuple(); +SELECT uuid FROM system.tables WHERE database = currentDatabase() and table = 'x'; +SELECT replica_path FROM system.replicas WHERE database = currentDatabase() and table = 'x'; +DROP TABLE x; From d7056d642d6d4d4e136646b18d6db37437dfd81b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Feb 2024 13:53:40 +0100 Subject: [PATCH 028/105] Addressed review comments --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 56 ++++++++++++----------- src/Analyzer/TableFunctionNode.h | 10 +++- src/Storages/StorageView.cpp | 1 + 3 files changed, 39 insertions(+), 28 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 02e70e53c5a..3d20242a70a 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6638,13 +6638,16 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, String database_name = scope.context->getCurrentDatabase(); String table_name; - if (table_function_node->getOriginalAST() && table_function_node->getOriginalAST()->as()) + auto function_ast = table_function_node->getOriginalAST() ? table_function_node->getOriginalAST()->as() : nullptr; + auto &table_function_node_typed = table_function_node->as(); + + if (function_ast) { - table_name = table_function_node->getOriginalAST()->as()->name; - if (table_function_node->getOriginalAST()->as()->is_compound_name) + table_name = function_ast->name; + if (function_ast->is_compound_name) { std::vector parts; - splitInto<'.'>(parts, table_function_node->getOriginalAST()->as()->name); + splitInto<'.'>(parts, function_ast->name); if (parts.size() == 2) { @@ -6652,35 +6655,34 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, table_name = parts[1]; } } - } - auto & table_function_node_typed = table_function_node->as(); - - StoragePtr table = table_name.empty() ? nullptr : DatabaseCatalog::instance().tryGetTable({database_name, table_name}, scope.context->getQueryContext()); - if (table) - { - if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) + StoragePtr table = table_name.empty() ? nullptr : DatabaseCatalog::instance().tryGetTable( + {database_name, table_name}, scope.context->getQueryContext()); + if (table) { - auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_function_node->getOriginalAST()); - StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); + if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) + { + auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues( + table_function_node->getOriginalAST()); + StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); - ASTCreateQuery create; - create.select = query->as(); - auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, scope.context); - auto res = std::make_shared(StorageID(database_name, table_name), - create, - ColumnsDescription(sample_block.getNamesAndTypesList()), - /* comment */ "", - /* is_parameterized_view */ true); - res->startup(); - table_function_node->getOriginalAST()->as()->prefer_subquery_to_function_formatting = true; - table_function_node_typed.resolve(std::move(res), scope.context); - return; + ASTCreateQuery create; + create.select = query->as(); + auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, scope.context); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); + res->startup(); + function_ast->prefer_subquery_to_function_formatting = true; + table_function_node_typed.resolve(std::move(res), scope.context); + return; + } } } - if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index ed1a26c4dd4..cb65511bfce 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -73,7 +74,14 @@ public: /// Returns true, if table function is resolved, false otherwise bool isResolved() const { - return storage != nullptr && table_function != nullptr; + /// For parameterized view, we only have storage + if (storage) + if (storage->as() && storage->as()->isParameterizedView()) + return true; + else + return table_function != nullptr; + else + return false; } /// Get table function, returns nullptr if table function node is not resolved diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 6b80e2450c4..588e5303117 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -114,6 +114,7 @@ StorageView::StorageView( StorageInMemoryMetadata storage_metadata; if (!is_parameterized_view_) { + /// If CREATE query is to create parameterized view, then we dont want to set columns if (!query.isParameterizedView()) storage_metadata.setColumns(columns_); } From 8f71d7817c60bec0a050437aa28cd4e28298daea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Feb 2024 14:57:45 +0100 Subject: [PATCH 029/105] Fix clang build --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 3d20242a70a..ed67250312a 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6638,7 +6638,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, String database_name = scope.context->getCurrentDatabase(); String table_name; - auto function_ast = table_function_node->getOriginalAST() ? table_function_node->getOriginalAST()->as() : nullptr; + auto * function_ast = table_function_node->getOriginalAST() ? table_function_node->getOriginalAST()->as() : nullptr; auto &table_function_node_typed = table_function_node->as(); if (function_ast) From 73cf923033fb4feec9653fd6d2bd7111c652baeb Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Feb 2024 18:03:00 +0000 Subject: [PATCH 030/105] Update test --- .../0_stateless/02500_numbers_inference.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02500_numbers_inference.sh b/tests/queries/0_stateless/02500_numbers_inference.sh index ce9cd5bdc9f..5d863bd616f 100755 --- a/tests/queries/0_stateless/02500_numbers_inference.sh +++ b/tests/queries/0_stateless/02500_numbers_inference.sh @@ -8,10 +8,10 @@ $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1.2}')"; echo '{"x" : 1.2}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1}')"; echo '{"x" : 1}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; -$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1e10}')"; -echo '{"x" : 1e10}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; -$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, 1, 1e10]}')"; -echo '{"x" : [1, 42.42, 1, 1e10]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; +$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1e10}')" --input_format_try_infer_exponent_floats=1; +echo '{"x" : 1e10}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1; +$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, 1, 1e10]}')" --input_format_try_infer_exponent_floats=1; +echo '{"x" : [1, 42.42, 1, 1e10]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, false]}')"; echo '{"x" : [1, 42.42, false]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; @@ -19,10 +19,10 @@ $CLICKHOUSE_LOCAL -q "desc format(TSV, '1.2')"; echo '1.2' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; $CLICKHOUSE_LOCAL -q "desc format(TSV, '1')"; echo '1' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; -$CLICKHOUSE_LOCAL -q "desc format(TSV, '1e10')"; -echo '1e10' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; -$CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, 1, 1e10]')"; -echo '[1, 42.42, 1, 1e10]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; +$CLICKHOUSE_LOCAL -q "desc format(TSV, '1e10')" --input_format_try_infer_exponent_floats=1; +echo '1e10' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1; +$CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, 1, 1e10]')" --input_format_try_infer_exponent_floats=1; +echo '[1, 42.42, 1, 1e10]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1; $CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, false]')"; echo '[1, 42.42, false]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; From ed5591e68cce58624bab515286a128352213377e Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 13 Feb 2024 18:20:57 +0000 Subject: [PATCH 031/105] changes due to review --- src/Functions/FunctionBinaryArithmetic.h | 78 ++++++++++-------------- 1 file changed, 31 insertions(+), 47 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 9b0afee5053..d2e74b0cb71 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -148,14 +148,7 @@ public: static constexpr bool allow_decimal = IsOperation::allow_decimal; static constexpr bool only_integer = IsOperation::div_int || IsOperation::div_int_or_zero; - /// Appropriate result type for binary operator on numeric types. "Date" can also mean - /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). - using ResultDataType = Switch< - /// Result must be Integer - Case::div_int || IsOperation::div_int_or_zero, DataTypeFromFieldType>, - - /// Decimal cases - Case || IsDataTypeDecimal), InvalidType>, + using DecimalResultType = Switch< Case< IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, LeftDataType>, @@ -163,6 +156,18 @@ public: Case && IsIntegralOrExtended, LeftDataType>, Case && IsIntegralOrExtended, RightDataType>, + /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) + Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, + Case && !IsIntegralOrExtendedOrDecimal, InvalidType>>; /// Determine result decimal type as it would be with usual division (as we determine BinaryOperationTraits::ResultType) + + /// Appropriate result type for binary operator on numeric types. "Date" can also mean + /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). + using ResultDataType = Switch< + /// Result must be Integer + Case::div_int || IsOperation::div_int_or_zero, DataTypeFromFieldType>, + /// Decimal cases + Case || IsDataTypeDecimal, DecimalResultType>, + /// e.g Decimal +-*/ Float, least(Decimal, Float), greatest(Decimal, Float) = Float64 Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, @@ -1669,26 +1674,23 @@ public: if constexpr (!std::is_same_v) { + if constexpr (is_div_int || is_div_int_or_zero) + type_res = std::make_shared(); if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { - if constexpr (is_div_int || is_div_int_or_zero) - type_res = std::make_shared(); - else + if constexpr (is_division) { - if constexpr (is_division) + if (context->getSettingsRef().decimal_check_overflow) { - if (context->getSettingsRef().decimal_check_overflow) - { - /// Check overflow by using operands scale (based on big decimal division implementation details): - /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers - /// i.e. int_operand = decimal_operand*10^scale - /// For division, left operand will be scaled by right operand scale also to do big integer division, - /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale - /// So, we can check upfront possible overflow just by checking max scale used for left operand - /// Note: it doesn't detect all possible overflow during big decimal division - if (left.getScale() + right.getScale() > ResultDataType::maxPrecision()) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); - } + /// Check overflow by using operands scale (based on big decimal division implementation details): + /// big decimal arithmetic is based on big integers, decimal operands are converted to big integers + /// i.e. int_operand = decimal_operand*10^scale + /// For division, left operand will be scaled by right operand scale also to do big integer division, + /// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale + /// So, we can check upfront possible overflow just by checking max scale used for left operand + /// Note: it doesn't detect all possible overflow during big decimal division + if (left.getScale() + right.getScale() > ResultDataType::maxPrecision()) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); } ResultDataType result_type = decimalResultType(left, right); type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); @@ -1697,24 +1699,15 @@ public: else if constexpr (((IsDataTypeDecimal && IsFloatingPoint) || (IsDataTypeDecimal && IsFloatingPoint))) { - if constexpr (is_div_int || is_div_int_or_zero) - type_res = std::make_shared(); - else - type_res = std::make_shared(); + type_res = std::make_shared(); } else if constexpr (IsDataTypeDecimal) { - if constexpr (is_div_int || is_div_int_or_zero) - type_res = std::make_shared(); - else - type_res = std::make_shared(left.getPrecision(), left.getScale()); + type_res = std::make_shared(left.getPrecision(), left.getScale()); } else if constexpr (IsDataTypeDecimal) { - if constexpr (is_div_int || is_div_int_or_zero) - type_res = std::make_shared(); - else - type_res = std::make_shared(right.getPrecision(), right.getScale()); + type_res = std::make_shared(right.getPrecision(), right.getScale()); } else if constexpr (std::is_same_v) { @@ -2024,6 +2017,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A using LeftDataType = std::decay_t; using RightDataType = std::decay_t; using ResultDataType = typename BinaryOperationTraits::ResultDataType; + using DecimalResultType = typename BinaryOperationTraits::DecimalResultType; if constexpr (std::is_same_v) return nullptr; @@ -2075,19 +2069,9 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A col_left_size, right_nullmap); } + /// Here we check if we have `intDiv` or `intDivOrZero` and at least one of the arguments is decimal, because in this case originally we had result as decimal, so we need to convert result into integer after calculations else if constexpr (!decimal_with_float && (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal)) { - using DecimalResultType = Switch< - Case< - IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, - LeftDataType>, - Case && IsDataTypeDecimal, RightDataType>, - Case && IsIntegralOrExtended, LeftDataType>, - Case && IsIntegralOrExtended, RightDataType>, - - /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) - Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, - Case && !IsIntegralOrExtendedOrDecimal, InvalidType>>; /// Determine result decimal type as it would be with usual division (as we determine BinaryOperationTraits::ResultType) if constexpr (!std::is_same_v) { From a27639a7f90c0575618ee4433c6545aa554c7750 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Feb 2024 22:06:00 +0000 Subject: [PATCH 032/105] Fix: call first exception callback on exception it'll update query_log --- src/Interpreters/executeQuery.cpp | 6 +++++- .../test_insert_exception_over_http/test.py | 12 ++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 345472fe4d0..6dd6d8eb6a0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1511,13 +1511,17 @@ void executeQuery( } catch (...) { + /// first execute on exception callback, it includes updating query_log + /// otherwise closing record ('ExceptionWhileProcessing') can be not appended in query_log + /// due to possible exceptions in functions called bellow (passed as parameter here) + streams.onException(); + if (handle_exception_in_output_format) { update_format_on_exception_if_needed(); if (output_format) handle_exception_in_output_format(*output_format); } - streams.onException(); throw; } diff --git a/tests/integration/test_insert_exception_over_http/test.py b/tests/integration/test_insert_exception_over_http/test.py index 962c20d6bff..826e0d01a81 100644 --- a/tests/integration/test_insert_exception_over_http/test.py +++ b/tests/integration/test_insert_exception_over_http/test.py @@ -25,13 +25,17 @@ def test_insert_exception_over_http(start_cluster): ) assert True == instance.http_query_and_get_error( - "insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)" + "insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)", method='POST' ).startswith("500 Internal Server Error") assert "0\n" == instance.query("select count() from tt") + instance.query("SYSTEM FLUSH LOGS") - assert "2\n" == instance.query( - "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday()" - ) + + assert "1\n" == instance.query( + "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'QueryStart'") + + assert "1\n" == instance.query( + "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'ExceptionWhileProcessing'") instance.query("DROP TABLE tt SYNC") From bc456820a0286d290f606ea5413e444614b9ddf7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Feb 2024 22:19:27 +0000 Subject: [PATCH 033/105] Automatic style fix --- .../integration/test_insert_exception_over_http/test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_insert_exception_over_http/test.py b/tests/integration/test_insert_exception_over_http/test.py index 826e0d01a81..a03d68e0b03 100644 --- a/tests/integration/test_insert_exception_over_http/test.py +++ b/tests/integration/test_insert_exception_over_http/test.py @@ -25,7 +25,8 @@ def test_insert_exception_over_http(start_cluster): ) assert True == instance.http_query_and_get_error( - "insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)", method='POST' + "insert into tt settings insert_keeper_max_retries=0, insert_keeper_fault_injection_probability=1.0, log_comment='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' values (1), (2), (3), (4), (5)", + method="POST", ).startswith("500 Internal Server Error") assert "0\n" == instance.query("select count() from tt") @@ -33,9 +34,11 @@ def test_insert_exception_over_http(start_cluster): instance.query("SYSTEM FLUSH LOGS") assert "1\n" == instance.query( - "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'QueryStart'") + "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'QueryStart'" + ) assert "1\n" == instance.query( - "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'ExceptionWhileProcessing'") + "select count() from system.query_log where log_comment ='02988_66a57d6f-d1cc-4693-8bf4-206848edab87' and current_database = currentDatabase() and event_date >= yesterday() and type = 'ExceptionWhileProcessing'" + ) instance.query("DROP TABLE tt SYNC") From 3d91b12a2fc29d44f2631a811e960ddc5936c922 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Feb 2024 22:34:10 +0000 Subject: [PATCH 034/105] Fix typo --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6dd6d8eb6a0..e8a032974c6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1513,7 +1513,7 @@ void executeQuery( { /// first execute on exception callback, it includes updating query_log /// otherwise closing record ('ExceptionWhileProcessing') can be not appended in query_log - /// due to possible exceptions in functions called bellow (passed as parameter here) + /// due to possible exceptions in functions called below (passed as parameter here) streams.onException(); if (handle_exception_in_output_format) From ac601a0cb9a8a81e0b690df317b9c7a98f17e1ad Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 13 Feb 2024 23:12:21 +0000 Subject: [PATCH 035/105] fix build --- src/Functions/FunctionBinaryArithmetic.h | 31 +++++++++++++++++------- 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 667854a622b..fa2ff8fc275 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -147,17 +147,24 @@ private: /// it's not correct for Decimal public: static constexpr bool allow_decimal = IsOperation::allow_decimal; - using DecimalResultType = Switch< - Case< - IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, - LeftDataType>, + using DecimalResultDataType = Switch< + Case, + Case && IsDataTypeDecimal && UseLeftDecimal, LeftDataType>, Case && IsDataTypeDecimal, RightDataType>, Case && IsIntegralOrExtended, LeftDataType>, Case && IsIntegralOrExtended, RightDataType>, - /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) + /// e.g Decimal +-*/ Float, least(Decimal, Float), greatest(Decimal, Float) = Float64 + Case && IsFloatingPoint, DataTypeFloat64>, + Case && IsFloatingPoint, DataTypeFloat64>, + + Case::bit_hamming_distance && IsIntegral && IsIntegral, DataTypeUInt8>, + Case::bit_hamming_distance && IsFixedString && IsFixedString, DataTypeUInt16>, + Case::bit_hamming_distance && IsString && IsString, DataTypeUInt64>, + + /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, - Case && !IsIntegralOrExtendedOrDecimal, InvalidType>>; /// Determine result decimal type as it would be with usual division (as we determine BinaryOperationTraits::ResultType) + Case && !IsIntegralOrExtendedOrDecimal, InvalidType>>; /// Appropriate result type for binary operator on numeric types. "Date" can also mean /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). @@ -165,7 +172,13 @@ public: /// Result must be Integer Case::div_int || IsOperation::div_int_or_zero, DataTypeFromFieldType>, /// Decimal cases - Case || IsDataTypeDecimal, DecimalResultType>, + Case || IsDataTypeDecimal, DecimalResultDataType>, + Case< + IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, + LeftDataType>, + Case && IsDataTypeDecimal, RightDataType>, + Case && IsIntegralOrExtended, LeftDataType>, + Case && IsIntegralOrExtended, RightDataType>, /// e.g Decimal +-*/ Float, least(Decimal, Float), greatest(Decimal, Float) = Float64 Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, @@ -1678,7 +1691,7 @@ public: { if constexpr (is_div_int || is_div_int_or_zero) type_res = std::make_shared(); - if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) + else if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { if constexpr (is_division) { @@ -2019,7 +2032,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A using LeftDataType = std::decay_t; using RightDataType = std::decay_t; using ResultDataType = typename BinaryOperationTraits::ResultDataType; - using DecimalResultType = typename BinaryOperationTraits::DecimalResultType; + using DecimalResultType = typename BinaryOperationTraits::DecimalResultDataType; if constexpr (std::is_same_v) return nullptr; From 59d8912f6361abc0df81e385cb1bf0f4e7cf42d1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Feb 2024 12:20:11 +0100 Subject: [PATCH 036/105] Update 02327_try_infer_integers_schema_inference test --- .../0_stateless/02327_try_infer_integers_schema_inference.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql index 0ceed178865..a4a69f4fa40 100644 --- a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql +++ b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql @@ -1,6 +1,7 @@ -- Tags: no-fasttest set input_format_try_infer_integers=1; +set input_format_try_infer_exponent_floats=1; select 'JSONEachRow'; desc format(JSONEachRow, '{"x" : 123}'); From fd5b5722259344c60ff6ec77ae0a58a0949d66a9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 14 Feb 2024 13:00:22 +0000 Subject: [PATCH 037/105] Fix --- src/Common/FailPoint.h | 1 + src/Interpreters/executeQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/FailPoint.h b/src/Common/FailPoint.h index a0081d67081..6aafbc0053a 100644 --- a/src/Common/FailPoint.h +++ b/src/Common/FailPoint.h @@ -1,5 +1,6 @@ #pragma once +#include "config.h" #include #include #include diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e8a032974c6..11eaffb99ff 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1380,7 +1380,7 @@ void executeQuery( result_details.content_type = output_format->getContentType(); result_details.format = format_name; - fiu_do_on(FailPoint::execute_query_calling_empty_set_result_func_on_exception, { + fiu_do_on(FailPoints::execute_query_calling_empty_set_result_func_on_exception, { set_result_details = nullptr; set_result_details(result_details); }); From 41304c3e11934e105dbb82dd805bc4414ab952ac Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 14 Feb 2024 14:08:20 +0100 Subject: [PATCH 038/105] Update src/Functions/FunctionBinaryArithmetic.h Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Functions/FunctionBinaryArithmetic.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index fa2ff8fc275..967c2b73881 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -644,7 +644,8 @@ private: { if ((*right_nullmap)[0]) { - c[0] = ResultType(); + for (size_t i = 0; i < size; ++i) + c[i] = ResultType(); return; } From eafa4219d9aeadd732c67372eef7aaf428afdc06 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 14 Feb 2024 13:17:55 +0000 Subject: [PATCH 039/105] Remove unnecessary include --- src/Common/FailPoint.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index e0639df29b4..a23133b7522 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -1,4 +1,3 @@ -#include "config.h" #include #include #include From d5ab7d8df08fecb0235416e7c5b15630cae4201b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 14 Feb 2024 13:19:34 +0000 Subject: [PATCH 040/105] Keep header the same as before --- src/Common/FailPoint.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FailPoint.h b/src/Common/FailPoint.h index 6aafbc0053a..613cfb15322 100644 --- a/src/Common/FailPoint.h +++ b/src/Common/FailPoint.h @@ -1,6 +1,6 @@ #pragma once - #include "config.h" + #include #include #include From c1897e74a83101130ffa369dd6c58993c17f5e6f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 14 Feb 2024 14:04:25 +0000 Subject: [PATCH 041/105] fix --- src/Functions/FunctionBinaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 967c2b73881..9b4249b0aef 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1708,9 +1708,9 @@ public: if (left.getScale() + right.getScale() > ResultDataType::maxPrecision()) throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division"); } - ResultDataType result_type = decimalResultType(left, right); - type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } + ResultDataType result_type = decimalResultType(left, right); + type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); } else if constexpr (((IsDataTypeDecimal && IsFloatingPoint) || (IsDataTypeDecimal && IsFloatingPoint))) From bbe38a3fe4717fad4c79b2ee5a8e40323c8e3dcb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 15 Feb 2024 09:15:51 +0100 Subject: [PATCH 042/105] Add ability to escape quotes in Values format with single quote Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 2 ++ src/DataTypes/Serializations/SerializationString.cpp | 7 +++++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../0_stateless/02993_values_escape_quote.reference | 3 +++ .../0_stateless/02993_values_escape_quote.sql | 12 ++++++++++++ 6 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02993_values_escape_quote.reference create mode 100644 tests/queries/0_stateless/02993_values_escape_quote.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 42dad28aa74..53ba18a3001 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1146,6 +1146,8 @@ class IColumn; M(Bool, output_format_sql_insert_use_replace, false, "Use REPLACE statement instead of INSERT", 0) \ M(Bool, output_format_sql_insert_quote_names, true, "Quote column names with '`' characters", 0) \ \ + M(Bool, output_format_values_escape_quote_with_quote, false, "If true escape ' with '', otherwise quoted with \\'", 0) \ + \ M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \ M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \ \ diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 6bffa0ff72e..fd46206e9ad 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -334,9 +334,12 @@ bool SerializationString::tryDeserializeTextEscaped(IColumn & column, ReadBuffer return read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); return true; }); } -void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const +void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeQuotedString(assert_cast(column).getDataAt(row_num), ostr); + if (settings.values.escape_quote_with_quote) + writeQuotedStringPostgreSQL(assert_cast(column).getDataAt(row_num).toView(), ostr); + else + writeQuotedString(assert_cast(column).getDataAt(row_num), ostr); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8c39b4b71e4..866b6934b70 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -181,6 +181,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; + format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote; format_settings.with_names_use_header = settings.input_format_with_names_use_header; format_settings.with_types_use_header = settings.input_format_with_types_use_header; format_settings.write_statistics = settings.output_format_write_statistics; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bdd2dda5287..cda15fd6531 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -361,6 +361,7 @@ struct FormatSettings bool deduce_templates_of_expressions = true; bool accurate_types_of_literals = true; bool allow_data_after_semicolon = false; + bool escape_quote_with_quote = false; } values; enum class ORCCompression diff --git a/tests/queries/0_stateless/02993_values_escape_quote.reference b/tests/queries/0_stateless/02993_values_escape_quote.reference new file mode 100644 index 00000000000..29d6a133fec --- /dev/null +++ b/tests/queries/0_stateless/02993_values_escape_quote.reference @@ -0,0 +1,3 @@ +('foo')('foo\'bar')('foo\'\'bar') +output_format_values_escape_quote_with_quote=1 +('foo')('foo''bar')('foo''''bar') diff --git a/tests/queries/0_stateless/02993_values_escape_quote.sql b/tests/queries/0_stateless/02993_values_escape_quote.sql new file mode 100644 index 00000000000..e6fc5f1b280 --- /dev/null +++ b/tests/queries/0_stateless/02993_values_escape_quote.sql @@ -0,0 +1,12 @@ +select 'foo' format Values; +select 'foo\'bar' format Values; +select 'foo\'\'bar' format Values; + +select '\noutput_format_values_escape_quote_with_quote=1' format LineAsString; +set output_format_values_escape_quote_with_quote=1; + +select 'foo' format Values; +select 'foo\'bar' format Values; +select 'foo\'\'bar' format Values; +-- fix no newline at end of file +select '' format LineAsString; From f10fc95933cb3beaf63329c97c412ec8201f082f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 15 Feb 2024 09:28:24 +0100 Subject: [PATCH 043/105] Fix INSERT into SQLite with single quote Previously it leads to syntax error, due to incorrect escaping of single quotes for SQLite, "\'" had been used instead of "''" So set output_format_values_escape_quote_with_quote=true for SQLite to fix this. v2: prepare modified Context for writing on storage creation Signed-off-by: Azat Khuzhin --- src/Storages/StorageSQLite.cpp | 17 ++++++++++++++++- src/Storages/StorageSQLite.h | 3 +++ .../01889_sqlite_read_write.reference | 2 +- .../0_stateless/01889_sqlite_read_write.sh | 2 +- 4 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 85c5e16a1bf..30cca409dc8 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -19,6 +19,20 @@ #include #include +namespace +{ + +using namespace DB; + +ContextPtr makeSQLiteWriteContext(ContextPtr context) +{ + auto write_context = Context::createCopy(context); + write_context->setSetting("output_format_values_escape_quote_with_quote", Field(true)); + return write_context; +} + +} + namespace DB { @@ -43,6 +57,7 @@ StorageSQLite::StorageSQLite( , database_path(database_path_) , sqlite_db(sqlite_db_) , log(getLogger("StorageSQLite (" + table_id_.table_name + ")")) + , write_context(makeSQLiteWriteContext(getContext())) { StorageInMemoryMetadata storage_metadata; @@ -144,7 +159,7 @@ public: sqlbuf << ") VALUES "; - auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext()); + auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.write_context); writer->write(block); sqlbuf << ";"; diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index baacdfb4899..ed673123fe0 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -47,10 +47,13 @@ public: const String & table); private: + friend class SQLiteSink; /// for write_context + String remote_table_name; String database_path; SQLitePtr sqlite_db; LoggerPtr log; + ContextPtr write_context; }; } diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.reference b/tests/queries/0_stateless/01889_sqlite_read_write.reference index 9f2b382e41e..e605693d95d 100644 --- a/tests/queries/0_stateless/01889_sqlite_read_write.reference +++ b/tests/queries/0_stateless/01889_sqlite_read_write.reference @@ -29,7 +29,7 @@ CREATE TABLE default.sqlite_table3\n(\n `col1` String,\n `col2` Int32\n)\n not a null 2 3 4 -line6 6 +line\'6 6 7 test table function line1 1 diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.sh b/tests/queries/0_stateless/01889_sqlite_read_write.sh index 02b9a649e94..fd0a1df20ac 100755 --- a/tests/queries/0_stateless/01889_sqlite_read_write.sh +++ b/tests/queries/0_stateless/01889_sqlite_read_write.sh @@ -76,7 +76,7 @@ ${CLICKHOUSE_CLIENT} --query='DROP TABLE IF EXISTS sqlite_table3' ${CLICKHOUSE_CLIENT} --query="CREATE TABLE sqlite_table3 (col1 String, col2 Int32) ENGINE = SQLite('${DB_PATH}', 'table3')" ${CLICKHOUSE_CLIENT} --query='SHOW CREATE TABLE sqlite_table3;' | sed -r 's/(.*SQLite)(.*)/\1/' -${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES ('line6', 6);" +${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES ('line\'6', 6);" ${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES (NULL, 7);" ${CLICKHOUSE_CLIENT} --query='SELECT * FROM sqlite_table3 ORDER BY col2' From c580cdb96e2aaa25dd0e6ed719db519afbfc5613 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 15 Feb 2024 12:55:43 +0000 Subject: [PATCH 044/105] Fix review comment --- src/Interpreters/executeQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 11eaffb99ff..5595e9801b2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1381,6 +1381,7 @@ void executeQuery( result_details.format = format_name; fiu_do_on(FailPoints::execute_query_calling_empty_set_result_func_on_exception, { + // it will throw std::bad_function_call set_result_details = nullptr; set_result_details(result_details); }); From a11b9553ca290c28e8014832bc4fa2aad6c3f6f6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 15 Feb 2024 17:16:27 +0100 Subject: [PATCH 045/105] Fix implementation in new analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 69 ++++++------------- src/Analyzer/TableFunctionNode.cpp | 7 -- src/Analyzer/TableFunctionNode.h | 13 +--- src/Interpreters/Context.cpp | 30 ++++++++ src/Interpreters/Context.h | 2 + .../02428_parameterized_view.reference | 1 + .../0_stateless/02428_parameterized_view.sh | 3 +- 7 files changed, 58 insertions(+), 67 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index ed67250312a..269094cbf63 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6634,55 +6634,8 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, QueryExpressionsAliasVisitor & expressions_visitor, bool nested_table_function) { - - String database_name = scope.context->getCurrentDatabase(); - String table_name; - - auto * function_ast = table_function_node->getOriginalAST() ? table_function_node->getOriginalAST()->as() : nullptr; auto &table_function_node_typed = table_function_node->as(); - if (function_ast) - { - table_name = function_ast->name; - if (function_ast->is_compound_name) - { - std::vector parts; - splitInto<'.'>(parts, function_ast->name); - - if (parts.size() == 2) - { - database_name = parts[0]; - table_name = parts[1]; - } - } - - StoragePtr table = table_name.empty() ? nullptr : DatabaseCatalog::instance().tryGetTable( - {database_name, table_name}, scope.context->getQueryContext()); - if (table) - { - if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) - { - auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues( - table_function_node->getOriginalAST()); - StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); - - ASTCreateQuery create; - create.select = query->as(); - auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, scope.context); - auto res = std::make_shared(StorageID(database_name, table_name), - create, - ColumnsDescription(sample_block.getNamesAndTypesList()), - /* comment */ "", - /* is_parameterized_view */ true); - res->startup(); - function_ast->prefer_subquery_to_function_formatting = true; - table_function_node_typed.resolve(std::move(res), scope.context); - return; - } - } - } - if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); @@ -6693,6 +6646,28 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().tryGet(table_function_name, scope_context); if (!table_function_ptr) { + String database_name = scope_context->getCurrentDatabase(); + String table_name; + + auto function_ast = table_function_node->toAST(); + Identifier table_identifier{table_function_name}; + if (table_identifier.getPartsSize() == 1) + { + table_name = table_identifier[0]; + } + else if (table_identifier.getPartsSize() == 2) + { + database_name = table_identifier[0]; + table_name = table_identifier[1]; + } + + auto parametrized_view_storage = scope_context->getQueryContext()->buildParametrizedViewStorage(function_ast, database_name, table_name); + if (parametrized_view_storage) + { + table_function_node = std::make_shared(parametrized_view_storage, scope_context); + return; + } + auto hints = TableFunctionFactory::instance().getHints(table_function_name); if (!hints.empty()) throw Exception(ErrorCodes::UNKNOWN_FUNCTION, diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index f4ffe7f4ee5..e5158a06373 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -36,13 +36,6 @@ void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePt unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_); } -void TableFunctionNode::resolve(StoragePtr storage_value, ContextPtr context) -{ - storage = std::move(storage_value); - storage_id = storage->getStorageID(); - storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); -} - const StorageID & TableFunctionNode::getStorageID() const { if (!storage) diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index cb65511bfce..69237ac8416 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -74,14 +73,7 @@ public: /// Returns true, if table function is resolved, false otherwise bool isResolved() const { - /// For parameterized view, we only have storage - if (storage) - if (storage->as() && storage->as()->isParameterizedView()) - return true; - else - return table_function != nullptr; - else - return false; + return storage != nullptr && table_function != nullptr; } /// Get table function, returns nullptr if table function node is not resolved @@ -108,9 +100,6 @@ public: /// Resolve table function with table function, storage and context void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector unresolved_arguments_indexes_); - /// Resolve table function as parameterized view with storage and context - void resolve(StoragePtr storage_value, ContextPtr context); - /// Get storage id, throws exception if function node is not resolved const StorageID & getStorageID() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0e5897e7306..1f11779a6aa 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -94,6 +94,7 @@ #include #include #include +#include #include #include #include @@ -1931,6 +1932,35 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } +StoragePtr Context::buildParametrizedViewStorage(const ASTPtr & table_expression, const String & database_name, const String & table_name) +{ + if (table_name.empty()) + return nullptr; + + StoragePtr original_view = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, getQueryContext()); + if (!original_view || !original_view->isView()) + return nullptr; + auto * storage_view = original_view->as(); + if (!storage_view || !storage_view->isParameterizedView()) + return nullptr; + + auto query = original_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); + + ASTCreateQuery create; + create.select = query->as(); + auto sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query, shared_from_this()); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); + res->startup(); + return res; +} + + void Context::addViewSource(const StoragePtr & storage) { if (view_source) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8d40ccb301b..cdd188faa48 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -718,6 +718,8 @@ public: /// Overload for the new analyzer. Structure inference is performed in QueryAnalysisPass. StoragePtr executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr); + StoragePtr buildParametrizedViewStorage(const ASTPtr & table_expression, const String & database_name, const String & table_name); + void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index 422fdaa4983..fd77e6ed8df 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -23,6 +23,7 @@ ERROR 20 20 ERROR +20 30 20 30 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 499b8697ffc..c6f0927db36 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -72,7 +72,8 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog VALUES $CLICKHOUSE_CLIENT -q "INSERT INTO ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog VALUES ('Paper', 20, 1)" $CLICKHOUSE_CLIENT -q "CREATE VIEW ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1 AS SELECT * FROM ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog WHERE Price={price:UInt64}" $CLICKHOUSE_CLIENT -q "SELECT Price FROM ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1(price=20)" -$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20)" 2>&1 | grep -Fq "UNKNOWN_FUNCTION" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -Fq "UNKNOWN_FUNCTION" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS allow_experimental_analyzer = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book2', 30, 8)" From dfaea604c375882fd9f4f553ef2576848d11c531 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 15 Feb 2024 17:27:26 +0100 Subject: [PATCH 046/105] Remove redundant includes --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 269094cbf63..a2a6c8e128d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -79,14 +79,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include namespace ProfileEvents { From 8397b856ae98d2a54dac50d301104e6813bd1b25 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 15 Feb 2024 17:28:54 +0100 Subject: [PATCH 047/105] Cleanup --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a2a6c8e128d..4d141ee962f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6626,7 +6626,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, QueryExpressionsAliasVisitor & expressions_visitor, bool nested_table_function) { - auto &table_function_node_typed = table_function_node->as(); + auto & table_function_node_typed = table_function_node->as(); if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); From 2f36c9e965b01ca68a795499bfbdbec020c47609 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Feb 2024 00:56:24 +0100 Subject: [PATCH 048/105] Maybe less memory usage for primary keys in memory --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0f82e00edff..9e264cba3f8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -842,6 +842,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < key_size; ++i) { + loaded_index[i]->shrinkToFit(); loaded_index[i]->protect(); if (loaded_index[i]->size() != marks_count) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data from index file {}(expected size: " diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index fcf9d5bd17d..8da9298b290 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -660,7 +660,7 @@ private: virtual void appendFilesOfIndexGranularity(Strings & files) const; - /// Loads index file. + /// Loads the index file. void loadIndex(); void appendFilesOfIndex(Strings & files) const; From 77c8f671cd258129af6f627293fca2e45e063a5f Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 15 Feb 2024 18:09:11 -0800 Subject: [PATCH 049/105] [Docs] Add info on using select_sequential_consistency for INSERTS in cloud --- docs/en/sql-reference/statements/insert-into.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index f9d93305071..2eebd62cbc6 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -204,6 +204,20 @@ Result: └─────┴───────────────────────┘ ``` +## Inserts into ClickHouse Cloud + +By default, services on ClickHouse Cloud provide multiple replicas for high availability. When you connect to a service, a connection is established to one of these replicas. + +After an `INSERT` succeeds, data is written to the underlying storage. However, it may take some time for replicas to receive these updates. Therefore, if you use a different connection that executes a `SELECT` query on one of these other replicas, the updated data may not yet be reflected. + +It is possible to use the `select_sequential_consistency` to force replicas to receive the latest updates. Here is an example of a SELECT query using this setting: + +```sql +SELECT .... SETTINGS select_sequential_consistency = 1; +``` + +Note that using `select_sequential_consistency` will increase the load on ClickHouse Keeper (used by ClickHouse Cloud internally) and may result in slower performance depending on the load on the service. We recommend against enabling this setting unless necessary. The recommended approach is to execute read/writes in the same session or to use a client driver that uses the native protocol (and thus supports sticky connections). + ## Performance Considerations `INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this: From 5b83e771ed70870374defeade7feba3118a6abb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Feb 2024 13:06:57 +0100 Subject: [PATCH 050/105] Print CPU flags at startup --- programs/server/Server.cpp | 15 +++++ src/Common/{CpuId.h => CPUID.h} | 91 +++++++++++++++--------------- src/Common/ProfileEvents.cpp | 8 +-- src/Common/TargetSpecific.cpp | 22 ++++---- src/Common/ThreadProfileEvents.cpp | 16 +++--- src/Functions/FunctionsJSON.h | 2 +- src/Functions/divide/divide.cpp | 6 +- 7 files changed, 87 insertions(+), 73 deletions(-) rename src/Common/{CpuId.h => CPUID.h} (68%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 53fc32663e7..b85bf6f814f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -712,6 +713,20 @@ try getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores std::thread::hardware_concurrency()); + String cpu_info; +#define COLLECT_FLAG(X) \ + if (CPU::have##X()) \ + { \ + if (!cpu_info.empty()) \ + cpu_info += ", "; \ + cpu_info += #X; \ + } + + CPU_ID_ENUMERATE(COLLECT_FLAG) +#undef COLLECT_FLAG + + LOG_INFO(log, "Available CPU instructions: {}", cpu_info); + sanityChecks(*this); // Initialize global thread pool. Do it before we fetch configs from zookeeper diff --git a/src/Common/CpuId.h b/src/Common/CPUID.h similarity index 68% rename from src/Common/CpuId.h rename to src/Common/CPUID.h index 1d15867289d..44b608ac1fe 100644 --- a/src/Common/CpuId.h +++ b/src/Common/CPUID.h @@ -11,7 +11,7 @@ namespace DB { -namespace Cpu +namespace CPU { #if (defined(__x86_64__) || defined(__i386__)) @@ -98,7 +98,7 @@ inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT OP(AMXTILE) \ OP(AMXINT8) -union CpuInfo +union CPUInfo { UInt32 info[4]; @@ -110,9 +110,9 @@ union CpuInfo UInt32 edx; } registers; - inline explicit CpuInfo(UInt32 op) noexcept { cpuid(op, info); } + inline explicit CPUInfo(UInt32 op) noexcept { cpuid(op, info); } - inline CpuInfo(UInt32 op, UInt32 sub_op) noexcept { cpuid(op, sub_op, info); } + inline CPUInfo(UInt32 op, UInt32 sub_op) noexcept { cpuid(op, sub_op, info); } }; #define DEF_NAME(X) inline bool have##X() noexcept; @@ -121,67 +121,67 @@ union CpuInfo bool haveRDTSCP() noexcept { - return (CpuInfo(0x80000001).registers.edx >> 27) & 1u; + return (CPUInfo(0x80000001).registers.edx >> 27) & 1u; } bool haveSSE() noexcept { - return (CpuInfo(0x1).registers.edx >> 25) & 1u; + return (CPUInfo(0x1).registers.edx >> 25) & 1u; } bool haveSSE2() noexcept { - return (CpuInfo(0x1).registers.edx >> 26) & 1u; + return (CPUInfo(0x1).registers.edx >> 26) & 1u; } bool haveSSE3() noexcept { - return CpuInfo(0x1).registers.ecx & 1u; + return CPUInfo(0x1).registers.ecx & 1u; } bool havePCLMUL() noexcept { - return (CpuInfo(0x1).registers.ecx >> 1) & 1u; + return (CPUInfo(0x1).registers.ecx >> 1) & 1u; } bool haveSSSE3() noexcept { - return (CpuInfo(0x1).registers.ecx >> 9) & 1u; + return (CPUInfo(0x1).registers.ecx >> 9) & 1u; } bool haveSSE41() noexcept { - return (CpuInfo(0x1).registers.ecx >> 19) & 1u; + return (CPUInfo(0x1).registers.ecx >> 19) & 1u; } bool haveSSE42() noexcept { - return (CpuInfo(0x1).registers.ecx >> 20) & 1u; + return (CPUInfo(0x1).registers.ecx >> 20) & 1u; } bool haveF16C() noexcept { - return (CpuInfo(0x1).registers.ecx >> 29) & 1u; + return (CPUInfo(0x1).registers.ecx >> 29) & 1u; } bool havePOPCNT() noexcept { - return (CpuInfo(0x1).registers.ecx >> 23) & 1u; + return (CPUInfo(0x1).registers.ecx >> 23) & 1u; } bool haveAES() noexcept { - return (CpuInfo(0x1).registers.ecx >> 25) & 1u; + return (CPUInfo(0x1).registers.ecx >> 25) & 1u; } bool haveXSAVE() noexcept { - return (CpuInfo(0x1).registers.ecx >> 26) & 1u; + return (CPUInfo(0x1).registers.ecx >> 26) & 1u; } bool haveOSXSAVE() noexcept { - return (CpuInfo(0x1).registers.ecx >> 27) & 1u; + return (CPUInfo(0x1).registers.ecx >> 27) & 1u; } bool haveAVX() noexcept @@ -191,7 +191,7 @@ bool haveAVX() noexcept // https://bugs.chromium.org/p/chromium/issues/detail?id=375968 return haveOSXSAVE() // implies haveXSAVE() && (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS - && ((CpuInfo(0x1).registers.ecx >> 28) & 1u); // AVX bit + && ((CPUInfo(0x1).registers.ecx >> 28) & 1u); // AVX bit #else return false; #endif @@ -199,22 +199,22 @@ bool haveAVX() noexcept bool haveFMA() noexcept { - return haveAVX() && ((CpuInfo(0x1).registers.ecx >> 12) & 1u); + return haveAVX() && ((CPUInfo(0x1).registers.ecx >> 12) & 1u); } bool haveAVX2() noexcept { - return haveAVX() && ((CpuInfo(0x7, 0).registers.ebx >> 5) & 1u); + return haveAVX() && ((CPUInfo(0x7, 0).registers.ebx >> 5) & 1u); } bool haveBMI1() noexcept { - return (CpuInfo(0x7, 0).registers.ebx >> 3) & 1u; + return (CPUInfo(0x7, 0).registers.ebx >> 3) & 1u; } bool haveBMI2() noexcept { - return (CpuInfo(0x7, 0).registers.ebx >> 8) & 1u; + return (CPUInfo(0x7, 0).registers.ebx >> 8) & 1u; } bool haveAVX512F() noexcept @@ -224,8 +224,8 @@ bool haveAVX512F() noexcept return haveOSXSAVE() // implies haveXSAVE() && (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS && ((our_xgetbv(0) >> 5) & 7u) == 7u // ZMM state is enabled by OS - && CpuInfo(0x0).registers.eax >= 0x7 // leaf 7 is present - && ((CpuInfo(0x7, 0).registers.ebx >> 16) & 1u); // AVX512F bit + && CPUInfo(0x0).registers.eax >= 0x7 // leaf 7 is present + && ((CPUInfo(0x7, 0).registers.ebx >> 16) & 1u); // AVX512F bit #else return false; #endif @@ -233,87 +233,87 @@ bool haveAVX512F() noexcept bool haveAVX512DQ() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 17) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 17) & 1u); } bool haveRDSEED() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 18) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 18) & 1u); } bool haveADX() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 19) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 19) & 1u); } bool haveAVX512IFMA() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 21) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 21) & 1u); } bool havePCOMMIT() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 22) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 22) & 1u); } bool haveCLFLUSHOPT() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 23) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 23) & 1u); } bool haveCLWB() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 24) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 24) & 1u); } bool haveAVX512PF() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 26) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 26) & 1u); } bool haveAVX512ER() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 27) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 27) & 1u); } bool haveAVX512CD() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 28) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 28) & 1u); } bool haveSHA() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 29) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 29) & 1u); } bool haveAVX512BW() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 30) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 30) & 1u); } bool haveAVX512VL() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 31) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 31) & 1u); } bool havePREFETCHWT1() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ecx >> 0) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ecx >> 0) & 1u); } bool haveAVX512VBMI() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 1) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ecx >> 1) & 1u); } bool haveAVX512VBMI2() noexcept { - return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 6) & 1u); + return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ecx >> 6) & 1u); } bool haveRDRAND() noexcept { - return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x1).registers.ecx >> 30) & 1u); + return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x1).registers.ecx >> 30) & 1u); } inline bool haveAMX() noexcept @@ -330,22 +330,22 @@ inline bool haveAMX() noexcept bool haveAMXBF16() noexcept { return haveAMX() - && ((CpuInfo(0x7, 0).registers.edx >> 22) & 1u); // AMX-BF16 bit + && ((CPUInfo(0x7, 0).registers.edx >> 22) & 1u); // AMX-BF16 bit } bool haveAMXTILE() noexcept { return haveAMX() - && ((CpuInfo(0x7, 0).registers.edx >> 24) & 1u); // AMX-TILE bit + && ((CPUInfo(0x7, 0).registers.edx >> 24) & 1u); // AMX-TILE bit } bool haveAMXINT8() noexcept { return haveAMX() - && ((CpuInfo(0x7, 0).registers.edx >> 25) & 1u); // AMX-INT8 bit + && ((CPUInfo(0x7, 0).registers.edx >> 25) & 1u); // AMX-INT8 bit } -struct CpuFlagsCache +struct CPUFlagsCache { #define DEF_NAME(X) static inline bool have_##X = have##X(); CPU_ID_ENUMERATE(DEF_NAME) @@ -354,4 +354,3 @@ struct CpuFlagsCache } } - diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index bdc5d2d88a8..679da441c76 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -310,7 +310,7 @@ The server successfully detected this situation and will download merged part fr M(ParallelReplicasStealingLeftoversMicroseconds, "Time spent collecting orphaned segments") \ M(ParallelReplicasCollectingOwnedSegmentsMicroseconds, "Time spent collecting segments meant by hash") \ \ - M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ + M(PerfCPUCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \ M(PerfCacheReferences, "Cache accesses. Usually, this indicates Last Level Cache accesses, but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \ M(PerfCacheMisses, "Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in conjunction with the PERFCOUNTHWCACHEREFERENCES event to calculate cache miss rates.") \ @@ -319,12 +319,12 @@ The server successfully detected this situation and will download merged part fr M(PerfBusCycles, "Bus cycles, which can be different from total cycles.") \ M(PerfStalledCyclesFrontend, "Stalled cycles during issue.") \ M(PerfStalledCyclesBackend, "Stalled cycles during retirement.") \ - M(PerfRefCpuCycles, "Total cycles; not affected by CPU frequency scaling.") \ + M(PerfRefCPUCycles, "Total cycles; not affected by CPU frequency scaling.") \ \ - M(PerfCpuClock, "The CPU clock, a high-resolution per-CPU timer") \ + M(PerfCPUClock, "The CPU clock, a high-resolution per-CPU timer") \ M(PerfTaskClock, "A clock count specific to the task that is running") \ M(PerfContextSwitches, "Number of context switches") \ - M(PerfCpuMigrations, "Number of times the process has migrated to a new CPU") \ + M(PerfCPUMigrations, "Number of times the process has migrated to a new CPU") \ M(PerfAlignmentFaults, "Number of alignment faults. These happen when unaligned memory accesses happen; the kernel can handle these but it reduces performance. This happens only on some architectures (never on x86).") \ M(PerfEmulationFaults, "Number of emulation faults. The kernel sometimes traps on unimplemented instructions and emulates them for user space. This can negatively impact performance.") \ M(PerfMinEnabledTime, "For all events, minimum time that an event was enabled. Used to track event multiplexing influence") \ diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index b115d3a8734..49f396c0926 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include namespace DB { @@ -9,25 +9,25 @@ namespace DB UInt32 getSupportedArchs() { UInt32 result = 0; - if (Cpu::CpuFlagsCache::have_SSE42) + if (CPU::CPUFlagsCache::have_SSE42) result |= static_cast(TargetArch::SSE42); - if (Cpu::CpuFlagsCache::have_AVX) + if (CPU::CPUFlagsCache::have_AVX) result |= static_cast(TargetArch::AVX); - if (Cpu::CpuFlagsCache::have_AVX2) + if (CPU::CPUFlagsCache::have_AVX2) result |= static_cast(TargetArch::AVX2); - if (Cpu::CpuFlagsCache::have_AVX512F) + if (CPU::CPUFlagsCache::have_AVX512F) result |= static_cast(TargetArch::AVX512F); - if (Cpu::CpuFlagsCache::have_AVX512BW) + if (CPU::CPUFlagsCache::have_AVX512BW) result |= static_cast(TargetArch::AVX512BW); - if (Cpu::CpuFlagsCache::have_AVX512VBMI) + if (CPU::CPUFlagsCache::have_AVX512VBMI) result |= static_cast(TargetArch::AVX512VBMI); - if (Cpu::CpuFlagsCache::have_AVX512VBMI2) + if (CPU::CPUFlagsCache::have_AVX512VBMI2) result |= static_cast(TargetArch::AVX512VBMI2); - if (Cpu::CpuFlagsCache::have_AMXBF16) + if (CPU::CPUFlagsCache::have_AMXBF16) result |= static_cast(TargetArch::AMXBF16); - if (Cpu::CpuFlagsCache::have_AMXTILE) + if (CPU::CPUFlagsCache::have_AMXTILE) result |= static_cast(TargetArch::AMXTILE); - if (Cpu::CpuFlagsCache::have_AMXINT8) + if (CPU::CPUFlagsCache::have_AMXINT8) result |= static_cast(TargetArch::AMXINT8); return result; } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 990151d73ff..40ea1f43449 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -36,7 +36,7 @@ namespace ProfileEvents extern const Event OSReadBytes; extern const Event OSWriteBytes; - extern const Event PerfCpuCycles; + extern const Event PerfCPUCycles; extern const Event PerfInstructions; extern const Event PerfCacheReferences; extern const Event PerfCacheMisses; @@ -45,12 +45,12 @@ namespace ProfileEvents extern const Event PerfBusCycles; extern const Event PerfStalledCyclesFrontend; extern const Event PerfStalledCyclesBackend; - extern const Event PerfRefCpuCycles; + extern const Event PerfRefCPUCycles; - extern const Event PerfCpuClock; + extern const Event PerfCPUClock; extern const Event PerfTaskClock; extern const Event PerfContextSwitches; - extern const Event PerfCpuMigrations; + extern const Event PerfCPUMigrations; extern const Event PerfAlignmentFaults; extern const Event PerfEmulationFaults; extern const Event PerfMinEnabledTime; @@ -218,7 +218,7 @@ thread_local PerfEventsCounters current_thread_counters; // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html static const PerfEventInfo raw_events_info[] = { - HARDWARE_EVENT(PERF_COUNT_HW_CPU_CYCLES, PerfCpuCycles), + HARDWARE_EVENT(PERF_COUNT_HW_CPU_CYCLES, PerfCPUCycles), HARDWARE_EVENT(PERF_COUNT_HW_INSTRUCTIONS, PerfInstructions), HARDWARE_EVENT(PERF_COUNT_HW_CACHE_REFERENCES, PerfCacheReferences), HARDWARE_EVENT(PERF_COUNT_HW_CACHE_MISSES, PerfCacheMisses), @@ -227,13 +227,13 @@ static const PerfEventInfo raw_events_info[] = { HARDWARE_EVENT(PERF_COUNT_HW_BUS_CYCLES, PerfBusCycles), HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, PerfStalledCyclesFrontend), HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, PerfStalledCyclesBackend), - HARDWARE_EVENT(PERF_COUNT_HW_REF_CPU_CYCLES, PerfRefCpuCycles), + HARDWARE_EVENT(PERF_COUNT_HW_REF_CPU_CYCLES, PerfRefCPUCycles), // `cpu-clock` is a bit broken according to this: https://stackoverflow.com/a/56967896 - SOFTWARE_EVENT(PERF_COUNT_SW_CPU_CLOCK, PerfCpuClock), + SOFTWARE_EVENT(PERF_COUNT_SW_CPU_CLOCK, PerfCPUClock), SOFTWARE_EVENT(PERF_COUNT_SW_TASK_CLOCK, PerfTaskClock), SOFTWARE_EVENT(PERF_COUNT_SW_CONTEXT_SWITCHES, PerfContextSwitches), - SOFTWARE_EVENT(PERF_COUNT_SW_CPU_MIGRATIONS, PerfCpuMigrations), + SOFTWARE_EVENT(PERF_COUNT_SW_CPU_MIGRATIONS, PerfCPUMigrations), SOFTWARE_EVENT(PERF_COUNT_SW_ALIGNMENT_FAULTS, PerfAlignmentFaults), SOFTWARE_EVENT(PERF_COUNT_SW_EMULATION_FAULTS, PerfEmulationFaults), diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 31a99475b63..2539fa1aeb4 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -5,7 +5,7 @@ #include -#include +#include #include #include diff --git a/src/Functions/divide/divide.cpp b/src/Functions/divide/divide.cpp index 6262d42a666..0708964c7d4 100644 --- a/src/Functions/divide/divide.cpp +++ b/src/Functions/divide/divide.cpp @@ -1,5 +1,5 @@ #include "divide.h" -#include +#include #if defined(__x86_64__) namespace SSE2 @@ -26,9 +26,9 @@ template void divideImpl(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size) { #if defined(__x86_64__) - if (DB::Cpu::CpuFlagsCache::have_AVX2) + if (DB::CPU::CPUFlagsCache::have_AVX2) AVX2::divideImpl(a_pos, b, c_pos, size); - else if (DB::Cpu::CpuFlagsCache::have_SSE2) + else if (DB::CPU::CPUFlagsCache::have_SSE2) SSE2::divideImpl(a_pos, b, c_pos, size); #else Generic::divideImpl(a_pos, b, c_pos, size); From 39f363ba6e7dc21c1ac9d8e1dddc92d677e5b302 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Feb 2024 13:09:07 +0100 Subject: [PATCH 051/105] Print CPU flags at startup --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b85bf6f814f..593c90e44b5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -725,7 +725,7 @@ try CPU_ID_ENUMERATE(COLLECT_FLAG) #undef COLLECT_FLAG - LOG_INFO(log, "Available CPU instructions: {}", cpu_info); + LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); sanityChecks(*this); From cb0ce2aaa94a818284ef3147603fa3441a737876 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 16 Feb 2024 15:26:08 +0000 Subject: [PATCH 052/105] Fix build --- src/Common/ThreadProfileEvents.cpp | 2 -- src/Coordination/KeeperConstants.cpp | 8 ++++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 40ea1f43449..6a63d484cd9 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -6,10 +6,8 @@ #include "ProcfsMetricsProvider.h" #include "hasLinuxCapability.h" -#include #include #include -#include #include #include diff --git a/src/Coordination/KeeperConstants.cpp b/src/Coordination/KeeperConstants.cpp index 2aa84b691c4..f788095334e 100644 --- a/src/Coordination/KeeperConstants.cpp +++ b/src/Coordination/KeeperConstants.cpp @@ -85,7 +85,7 @@ M(OSReadChars) \ M(OSWriteChars) \ \ - M(PerfCpuCycles) \ + M(PerfCPUCycles) \ M(PerfInstructions) \ M(PerfCacheReferences) \ M(PerfCacheMisses) \ @@ -94,12 +94,12 @@ M(PerfBusCycles) \ M(PerfStalledCyclesFrontend) \ M(PerfStalledCyclesBackend) \ - M(PerfRefCpuCycles) \ + M(PerfRefCPUCycles) \ \ - M(PerfCpuClock) \ + M(PerfCPUClock) \ M(PerfTaskClock) \ M(PerfContextSwitches) \ - M(PerfCpuMigrations) \ + M(PerfCPUMigrations) \ M(PerfAlignmentFaults) \ M(PerfEmulationFaults) \ M(PerfMinEnabledTime) \ From 96d978243939536ba7877e9daa2e2a309d1aa5a3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 16 Feb 2024 19:16:02 +0300 Subject: [PATCH 053/105] Temporary table already exists exception message fix --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fdd18c9bdeb..d6d0ed317d5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1534,7 +1534,7 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder & std::lock_guard lock(mutex); if (external_tables_mapping.end() != external_tables_mapping.find(table_name)) - throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} already exists.", backQuoteIfNeed(table_name)); + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} already exists", backQuoteIfNeed(table_name)); external_tables_mapping.emplace(table_name, std::make_shared(std::move(temporary_table))); } From 8c66db0d1c7e4393fbc528c85964ed4e7ebdb22e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Feb 2024 19:21:25 +0000 Subject: [PATCH 054/105] Use max_query_size from context in DDLLogEntry instead of hardcoded 4096 --- src/Interpreters/DDLTask.cpp | 5 +++-- src/Interpreters/DDLTask.h | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 90eec421abf..db45fa0d842 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -74,6 +74,8 @@ void DDLLogEntry::setSettingsIfRequired(ContextPtr context) if (version >= SETTINGS_IN_ZK_VERSION) settings.emplace(context->getSettingsRef().changes()); + + max_query_size = context->getSettingsRef().max_query_size; } String DDLLogEntry::toString() const @@ -148,9 +150,8 @@ void DDLLogEntry::parse(const String & data) String settings_str; rb >> "settings: " >> settings_str >> "\n"; ParserSetQuery parser{true}; - constexpr UInt64 max_size = 4096; constexpr UInt64 max_depth = 16; - ASTPtr settings_ast = parseQuery(parser, settings_str, max_size, max_depth); + ASTPtr settings_ast = parseQuery(parser, settings_str, max_query_size, max_depth); settings.emplace(std::move(settings_ast->as()->changes)); } } diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 5a8a5bfb184..a1f20576281 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -89,6 +89,7 @@ struct DDLLogEntry OpenTelemetry::TracingContext tracing_context; String initial_query_id; bool is_backup_restore = false; + size_t max_query_size = DBMS_DEFAULT_MAX_QUERY_SIZE; void setSettingsIfRequired(ContextPtr context); String toString() const; From 4e6f04ddf1dbfcf925d0fc4a978e926b4fd7d358 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Feb 2024 19:30:16 +0000 Subject: [PATCH 055/105] Address comments --- src/Interpreters/DDLTask.cpp | 4 +--- src/Interpreters/DDLTask.h | 1 - 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index db45fa0d842..543d8b16791 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -74,8 +74,6 @@ void DDLLogEntry::setSettingsIfRequired(ContextPtr context) if (version >= SETTINGS_IN_ZK_VERSION) settings.emplace(context->getSettingsRef().changes()); - - max_query_size = context->getSettingsRef().max_query_size; } String DDLLogEntry::toString() const @@ -151,7 +149,7 @@ void DDLLogEntry::parse(const String & data) rb >> "settings: " >> settings_str >> "\n"; ParserSetQuery parser{true}; constexpr UInt64 max_depth = 16; - ASTPtr settings_ast = parseQuery(parser, settings_str, max_query_size, max_depth); + ASTPtr settings_ast = parseQuery(parser, settings_str, Context::getGlobalContextInstance()->getSettingsRef().max_query_size, max_depth); settings.emplace(std::move(settings_ast->as()->changes)); } } diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index a1f20576281..5a8a5bfb184 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -89,7 +89,6 @@ struct DDLLogEntry OpenTelemetry::TracingContext tracing_context; String initial_query_id; bool is_backup_restore = false; - size_t max_query_size = DBMS_DEFAULT_MAX_QUERY_SIZE; void setSettingsIfRequired(ContextPtr context); String toString() const; From b5ef034697188f7c2d61723800f718c55864193e Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Feb 2024 22:31:22 +0300 Subject: [PATCH 056/105] Implement system.dns_cache table (#59856) * system.dns_cache table WIP * system.dns_cache table * rollback unexpected contrib changes * rollback unexpected changes * add docs * code style fixes * optimize dns cache exporting * stateful test for system.dns_cache table * stateful test for system.dns_cache table * cr fixes * stateful test for system.dns_cache table * stateful test for system.dns_cache table * Update 02981_system_dns_cache_table.sql * Update 02981_system_dns_cache_table.reference --------- Co-authored-by: pufit --- docs/en/operations/system-tables/dns_cache.md | 34 ++++++++++ src/Common/DNSResolver.cpp | 26 +++++--- src/Common/DNSResolver.h | 9 ++- src/Storages/System/StorageSystemDNSCache.cpp | 65 +++++++++++++++++++ src/Storages/System/StorageSystemDNSCache.h | 25 +++++++ src/Storages/System/attachSystemTables.cpp | 2 + .../02981_system_dns_cache_table.reference | 2 + .../02981_system_dns_cache_table.sql | 2 + 8 files changed, 156 insertions(+), 9 deletions(-) create mode 100644 docs/en/operations/system-tables/dns_cache.md create mode 100644 src/Storages/System/StorageSystemDNSCache.cpp create mode 100644 src/Storages/System/StorageSystemDNSCache.h create mode 100644 tests/queries/0_stateless/02981_system_dns_cache_table.reference create mode 100644 tests/queries/0_stateless/02981_system_dns_cache_table.sql diff --git a/docs/en/operations/system-tables/dns_cache.md b/docs/en/operations/system-tables/dns_cache.md new file mode 100644 index 00000000000..ed5a5cd506f --- /dev/null +++ b/docs/en/operations/system-tables/dns_cache.md @@ -0,0 +1,34 @@ +--- +slug: /en/operations/system-tables/dns_cache +--- +# dns_cache + +Contains information about cached DNS records. + +Columns: + +- `hostname` ([String](../../sql-reference/data-types/string.md)) — cached hostname +- `ip_address` ([String](../../sql-reference/data-types/string.md)) — ip address for the hostname +- `family` ([String](../../sql-reference/data-types/string.md)) — family of the ip address: `IPv4`, `IPv6`, or `UNIX_LOCAL`. +- `cached_at` ([String](../../sql-reference/data-types/datetime.md)) - datetime when the record was cached + +**Example** + +Query: + +```sql +SELECT * FROM system.dns_cache; +``` + +Result: + +| hostname | ip\_address | ip\_family | cached\_at | +| :--- | :--- | :--- | :--- | +| localhost | ::1 | IPv6 | 2024-02-11 17:04:40 | +| localhost | 127.0.0.1 | IPv4 | 2024-02-11 17:04:40 | + +**See also** + +- [disable_internal_dns_cache setting](../../operations/server-configuration-parameters/settings.md#disable_internal_dns_cache) +- [dns_cache_update_period setting](../../operations/server-configuration-parameters/settings.md#dns_cache_update_period) +- [dns_max_consecutive_failures setting](../../operations/server-configuration-parameters/settings.md#dns_max_consecutive_failures) \ No newline at end of file diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index fcbbaf6b0be..70646d46434 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -4,13 +4,10 @@ #include #include #include -#include -#include #include #include #include #include -#include #include #include #include @@ -141,10 +138,10 @@ DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase & cache, const std::string & host) +DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase & cache, const std::string & host) { - auto [result, _ ] = cache.getOrSet(host, [&host]() { return std::make_shared(resolveIPAddressImpl(host)); }); - return *result; + auto [result, _ ] = cache.getOrSet(host, [&host]() {return std::make_shared(resolveIPAddressImpl(host), std::chrono::system_clock::now());}); + return result->addresses; } std::unordered_set reverseResolveImpl(const Poco::Net::IPAddress & address) @@ -179,7 +176,7 @@ struct DNSResolver::Impl using HostWithConsecutiveFailures = std::unordered_map; using AddressWithConsecutiveFailures = std::unordered_map; - CacheBase cache_host{100}; + CacheBase cache_host{100}; CacheBase> cache_address{100}; std::mutex drop_mutex; @@ -411,7 +408,7 @@ bool DNSResolver::updateHost(const String & host) const auto old_value = resolveIPAddressWithCache(impl->cache_host, host); auto new_value = resolveIPAddressImpl(host); const bool result = old_value != new_value; - impl->cache_host.set(host, std::make_shared(std::move(new_value))); + impl->cache_host.set(host, std::make_shared(std::move(new_value), std::chrono::system_clock::now())); return result; } @@ -438,6 +435,19 @@ void DNSResolver::addToNewAddresses(const Poco::Net::IPAddress & address) impl->new_addresses.insert({address, consecutive_failures}); } +std::vector> DNSResolver::cacheEntries() const +{ + std::lock_guard lock(impl->drop_mutex); + std::vector> entries; + + for (auto & [key, entry] : impl->cache_host.dump()) + { + entries.emplace_back(std::move(key), *entry); + } + + return entries; +} + DNSResolver::~DNSResolver() = default; DNSResolver & DNSResolver::instance() diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 965688f84f2..38f19791051 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -20,7 +20,11 @@ class DNSResolver : private boost::noncopyable { public: using IPAddresses = std::vector; - using IPAddressesPtr = std::shared_ptr; + using CacheEntry = struct + { + IPAddresses addresses; + std::chrono::system_clock::time_point cached_at; + }; static DNSResolver & instance(); @@ -58,6 +62,9 @@ public: /// Returns true if IP of any host has been changed or an element was dropped (too many failures) bool updateCache(UInt32 max_consecutive_failures); + /// Returns a copy of cache entries + std::vector> cacheEntries() const; + ~DNSResolver(); private: diff --git a/src/Storages/System/StorageSystemDNSCache.cpp b/src/Storages/System/StorageSystemDNSCache.cpp new file mode 100644 index 00000000000..828c120fcb6 --- /dev/null +++ b/src/Storages/System/StorageSystemDNSCache.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "StorageSystemDatabases.h" + +namespace DB +{ + +ColumnsDescription StorageSystemDNSCache::getColumnsDescription() +{ + return ColumnsDescription + { + {"hostname", std::make_shared(), "Hostname."}, + {"ip_address", std::make_shared(), "IP address."}, + {"ip_family", std::make_shared(), "IP address family."}, + {"cached_at", std::make_shared(), "Record cached timestamp."}, + }; +} + +void StorageSystemDNSCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + using HostIPPair = std::pair; + std::set reported_elements; + + for (const auto & [hostname, entry] : DNSResolver::instance().cacheEntries()) + { + for (const auto &address : entry.addresses) + { + std::string ip = address.toString(); + + // Cache might report the same ip address multiple times. Report only one of them. + if (reported_elements.contains(HostIPPair(hostname, ip))) + continue; + + reported_elements.insert(HostIPPair(hostname, ip)); + + std::string family_str; + switch (address.family()) + { + case Poco::Net::AddressFamily::IPv4: + family_str = "IPv4"; + break; + case Poco::Net::AddressFamily::IPv6: + family_str = "IPv6"; + break; + case Poco::Net::AddressFamily::UNIX_LOCAL: + family_str = "UNIX_LOCAL"; + break; + } + + size_t i = 0; + res_columns[i++]->insert(hostname); + res_columns[i++]->insert(ip); + res_columns[i++]->insert(family_str); + res_columns[i++]->insert(static_cast(std::chrono::system_clock::to_time_t(entry.cached_at))); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemDNSCache.h b/src/Storages/System/StorageSystemDNSCache.h new file mode 100644 index 00000000000..340060335b8 --- /dev/null +++ b/src/Storages/System/StorageSystemDNSCache.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +/// system.dns_cache table. +class StorageSystemDNSCache final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemDNSCache"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index bf898f57833..f08d6972b98 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -90,6 +90,7 @@ #include #include #include +#include #if defined(__ELF__) && !defined(OS_FREEBSD) #include @@ -166,6 +167,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dropped_tables", "Contains a list of tables which were dropped from Atomic databases but not completely removed yet."); attach(context, system_database, "dropped_tables_parts", "Contains parts of system.dropped_tables tables "); attach(context, system_database, "scheduler", "Contains information and status for scheduling nodes residing on the local server."); + attach(context, system_database, "dns_cache", "Contains information about cached DNS records."); #if defined(__ELF__) && !defined(OS_FREEBSD) attach(context, system_database, "symbols", "Contains information for introspection of ClickHouse binary. This table is only useful for C++ experts and ClickHouse engineers."); #endif diff --git a/tests/queries/0_stateless/02981_system_dns_cache_table.reference b/tests/queries/0_stateless/02981_system_dns_cache_table.reference new file mode 100644 index 00000000000..0e51a855bbe --- /dev/null +++ b/tests/queries/0_stateless/02981_system_dns_cache_table.reference @@ -0,0 +1,2 @@ +Ok. +localhost 127.0.0.1 IPv4 1 diff --git a/tests/queries/0_stateless/02981_system_dns_cache_table.sql b/tests/queries/0_stateless/02981_system_dns_cache_table.sql new file mode 100644 index 00000000000..87bb91ca29a --- /dev/null +++ b/tests/queries/0_stateless/02981_system_dns_cache_table.sql @@ -0,0 +1,2 @@ +SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers()); +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'; From 731c484b3d03edb7e1d62f5d97913b5658cfe375 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 01:46:41 +0100 Subject: [PATCH 057/105] Revert "Implement system.dns_cache table (#59856)" This reverts commit b5ef034697188f7c2d61723800f718c55864193e. --- docs/en/operations/system-tables/dns_cache.md | 34 ---------- src/Common/DNSResolver.cpp | 26 +++----- src/Common/DNSResolver.h | 9 +-- src/Storages/System/StorageSystemDNSCache.cpp | 65 ------------------- src/Storages/System/StorageSystemDNSCache.h | 25 ------- src/Storages/System/attachSystemTables.cpp | 2 - .../02981_system_dns_cache_table.reference | 2 - .../02981_system_dns_cache_table.sql | 2 - 8 files changed, 9 insertions(+), 156 deletions(-) delete mode 100644 docs/en/operations/system-tables/dns_cache.md delete mode 100644 src/Storages/System/StorageSystemDNSCache.cpp delete mode 100644 src/Storages/System/StorageSystemDNSCache.h delete mode 100644 tests/queries/0_stateless/02981_system_dns_cache_table.reference delete mode 100644 tests/queries/0_stateless/02981_system_dns_cache_table.sql diff --git a/docs/en/operations/system-tables/dns_cache.md b/docs/en/operations/system-tables/dns_cache.md deleted file mode 100644 index ed5a5cd506f..00000000000 --- a/docs/en/operations/system-tables/dns_cache.md +++ /dev/null @@ -1,34 +0,0 @@ ---- -slug: /en/operations/system-tables/dns_cache ---- -# dns_cache - -Contains information about cached DNS records. - -Columns: - -- `hostname` ([String](../../sql-reference/data-types/string.md)) — cached hostname -- `ip_address` ([String](../../sql-reference/data-types/string.md)) — ip address for the hostname -- `family` ([String](../../sql-reference/data-types/string.md)) — family of the ip address: `IPv4`, `IPv6`, or `UNIX_LOCAL`. -- `cached_at` ([String](../../sql-reference/data-types/datetime.md)) - datetime when the record was cached - -**Example** - -Query: - -```sql -SELECT * FROM system.dns_cache; -``` - -Result: - -| hostname | ip\_address | ip\_family | cached\_at | -| :--- | :--- | :--- | :--- | -| localhost | ::1 | IPv6 | 2024-02-11 17:04:40 | -| localhost | 127.0.0.1 | IPv4 | 2024-02-11 17:04:40 | - -**See also** - -- [disable_internal_dns_cache setting](../../operations/server-configuration-parameters/settings.md#disable_internal_dns_cache) -- [dns_cache_update_period setting](../../operations/server-configuration-parameters/settings.md#dns_cache_update_period) -- [dns_max_consecutive_failures setting](../../operations/server-configuration-parameters/settings.md#dns_max_consecutive_failures) \ No newline at end of file diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 70646d46434..fcbbaf6b0be 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -4,10 +4,13 @@ #include #include #include +#include +#include #include #include #include #include +#include #include #include #include @@ -138,10 +141,10 @@ DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase & cache, const std::string & host) +DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase & cache, const std::string & host) { - auto [result, _ ] = cache.getOrSet(host, [&host]() {return std::make_shared(resolveIPAddressImpl(host), std::chrono::system_clock::now());}); - return result->addresses; + auto [result, _ ] = cache.getOrSet(host, [&host]() { return std::make_shared(resolveIPAddressImpl(host)); }); + return *result; } std::unordered_set reverseResolveImpl(const Poco::Net::IPAddress & address) @@ -176,7 +179,7 @@ struct DNSResolver::Impl using HostWithConsecutiveFailures = std::unordered_map; using AddressWithConsecutiveFailures = std::unordered_map; - CacheBase cache_host{100}; + CacheBase cache_host{100}; CacheBase> cache_address{100}; std::mutex drop_mutex; @@ -408,7 +411,7 @@ bool DNSResolver::updateHost(const String & host) const auto old_value = resolveIPAddressWithCache(impl->cache_host, host); auto new_value = resolveIPAddressImpl(host); const bool result = old_value != new_value; - impl->cache_host.set(host, std::make_shared(std::move(new_value), std::chrono::system_clock::now())); + impl->cache_host.set(host, std::make_shared(std::move(new_value))); return result; } @@ -435,19 +438,6 @@ void DNSResolver::addToNewAddresses(const Poco::Net::IPAddress & address) impl->new_addresses.insert({address, consecutive_failures}); } -std::vector> DNSResolver::cacheEntries() const -{ - std::lock_guard lock(impl->drop_mutex); - std::vector> entries; - - for (auto & [key, entry] : impl->cache_host.dump()) - { - entries.emplace_back(std::move(key), *entry); - } - - return entries; -} - DNSResolver::~DNSResolver() = default; DNSResolver & DNSResolver::instance() diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 38f19791051..965688f84f2 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -20,11 +20,7 @@ class DNSResolver : private boost::noncopyable { public: using IPAddresses = std::vector; - using CacheEntry = struct - { - IPAddresses addresses; - std::chrono::system_clock::time_point cached_at; - }; + using IPAddressesPtr = std::shared_ptr; static DNSResolver & instance(); @@ -62,9 +58,6 @@ public: /// Returns true if IP of any host has been changed or an element was dropped (too many failures) bool updateCache(UInt32 max_consecutive_failures); - /// Returns a copy of cache entries - std::vector> cacheEntries() const; - ~DNSResolver(); private: diff --git a/src/Storages/System/StorageSystemDNSCache.cpp b/src/Storages/System/StorageSystemDNSCache.cpp deleted file mode 100644 index 828c120fcb6..00000000000 --- a/src/Storages/System/StorageSystemDNSCache.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include "StorageSystemDatabases.h" - -namespace DB -{ - -ColumnsDescription StorageSystemDNSCache::getColumnsDescription() -{ - return ColumnsDescription - { - {"hostname", std::make_shared(), "Hostname."}, - {"ip_address", std::make_shared(), "IP address."}, - {"ip_family", std::make_shared(), "IP address family."}, - {"cached_at", std::make_shared(), "Record cached timestamp."}, - }; -} - -void StorageSystemDNSCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const -{ - using HostIPPair = std::pair; - std::set reported_elements; - - for (const auto & [hostname, entry] : DNSResolver::instance().cacheEntries()) - { - for (const auto &address : entry.addresses) - { - std::string ip = address.toString(); - - // Cache might report the same ip address multiple times. Report only one of them. - if (reported_elements.contains(HostIPPair(hostname, ip))) - continue; - - reported_elements.insert(HostIPPair(hostname, ip)); - - std::string family_str; - switch (address.family()) - { - case Poco::Net::AddressFamily::IPv4: - family_str = "IPv4"; - break; - case Poco::Net::AddressFamily::IPv6: - family_str = "IPv6"; - break; - case Poco::Net::AddressFamily::UNIX_LOCAL: - family_str = "UNIX_LOCAL"; - break; - } - - size_t i = 0; - res_columns[i++]->insert(hostname); - res_columns[i++]->insert(ip); - res_columns[i++]->insert(family_str); - res_columns[i++]->insert(static_cast(std::chrono::system_clock::to_time_t(entry.cached_at))); - } - } -} - -} diff --git a/src/Storages/System/StorageSystemDNSCache.h b/src/Storages/System/StorageSystemDNSCache.h deleted file mode 100644 index 340060335b8..00000000000 --- a/src/Storages/System/StorageSystemDNSCache.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; - -/// system.dns_cache table. -class StorageSystemDNSCache final : public IStorageSystemOneBlock -{ -public: - std::string getName() const override { return "SystemDNSCache"; } - - static ColumnsDescription getColumnsDescription(); - -protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; - - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; -}; - -} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index f08d6972b98..bf898f57833 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -90,7 +90,6 @@ #include #include #include -#include #if defined(__ELF__) && !defined(OS_FREEBSD) #include @@ -167,7 +166,6 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dropped_tables", "Contains a list of tables which were dropped from Atomic databases but not completely removed yet."); attach(context, system_database, "dropped_tables_parts", "Contains parts of system.dropped_tables tables "); attach(context, system_database, "scheduler", "Contains information and status for scheduling nodes residing on the local server."); - attach(context, system_database, "dns_cache", "Contains information about cached DNS records."); #if defined(__ELF__) && !defined(OS_FREEBSD) attach(context, system_database, "symbols", "Contains information for introspection of ClickHouse binary. This table is only useful for C++ experts and ClickHouse engineers."); #endif diff --git a/tests/queries/0_stateless/02981_system_dns_cache_table.reference b/tests/queries/0_stateless/02981_system_dns_cache_table.reference deleted file mode 100644 index 0e51a855bbe..00000000000 --- a/tests/queries/0_stateless/02981_system_dns_cache_table.reference +++ /dev/null @@ -1,2 +0,0 @@ -Ok. -localhost 127.0.0.1 IPv4 1 diff --git a/tests/queries/0_stateless/02981_system_dns_cache_table.sql b/tests/queries/0_stateless/02981_system_dns_cache_table.sql deleted file mode 100644 index 87bb91ca29a..00000000000 --- a/tests/queries/0_stateless/02981_system_dns_cache_table.sql +++ /dev/null @@ -1,2 +0,0 @@ -SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers()); -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'; From 4c2ee234299f26d3716c0e952a78147a49faa651 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 01:50:20 +0100 Subject: [PATCH 058/105] Update insert-into.md --- docs/en/sql-reference/statements/insert-into.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 2eebd62cbc6..f5544f96750 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -210,7 +210,7 @@ By default, services on ClickHouse Cloud provide multiple replicas for high avai After an `INSERT` succeeds, data is written to the underlying storage. However, it may take some time for replicas to receive these updates. Therefore, if you use a different connection that executes a `SELECT` query on one of these other replicas, the updated data may not yet be reflected. -It is possible to use the `select_sequential_consistency` to force replicas to receive the latest updates. Here is an example of a SELECT query using this setting: +It is possible to use the `select_sequential_consistency` to force the replica to receive the latest updates. Here is an example of a SELECT query using this setting: ```sql SELECT .... SETTINGS select_sequential_consistency = 1; From 509d8ee99ca259db0233fe4c174541753169256d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 01:54:54 +0100 Subject: [PATCH 059/105] Apply review comments --- programs/server/Server.cpp | 2 ++ src/Common/CPUID.h | 14 +++++++------- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 593c90e44b5..0a3c23d746a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -713,6 +713,7 @@ try getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores std::thread::hardware_concurrency()); +#if defined(__x86_64__) String cpu_info; #define COLLECT_FLAG(X) \ if (CPU::have##X()) \ @@ -726,6 +727,7 @@ try #undef COLLECT_FLAG LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); +#endif sanityChecks(*this); diff --git a/src/Common/CPUID.h b/src/Common/CPUID.h index 44b608ac1fe..b47e7e808d7 100644 --- a/src/Common/CPUID.h +++ b/src/Common/CPUID.h @@ -2,7 +2,7 @@ #include -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) #include #endif @@ -14,7 +14,7 @@ namespace DB namespace CPU { -#if (defined(__x86_64__) || defined(__i386__)) +#if (defined(__x86_64__)) /// Our version is independent of -mxsave option, because we do dynamic dispatch. inline UInt64 our_xgetbv(UInt32 xcr) noexcept { @@ -30,7 +30,7 @@ inline UInt64 our_xgetbv(UInt32 xcr) noexcept inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept /// NOLINT { -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) __cpuid_count(op, sub_op, res[0], res[1], res[2], res[3]); return true; #else @@ -45,7 +45,7 @@ inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept /// NOLINT inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT { -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) __cpuid(op, res[0], res[1], res[2], res[3]); return true; #else @@ -186,7 +186,7 @@ bool haveOSXSAVE() noexcept bool haveAVX() noexcept { -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) // http://www.intel.com/content/dam/www/public/us/en/documents/manuals/64-ia-32-architectures-optimization-manual.pdf // https://bugs.chromium.org/p/chromium/issues/detail?id=375968 return haveOSXSAVE() // implies haveXSAVE() @@ -219,7 +219,7 @@ bool haveBMI2() noexcept bool haveAVX512F() noexcept { -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) // https://software.intel.com/en-us/articles/how-to-detect-knl-instruction-support return haveOSXSAVE() // implies haveXSAVE() && (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS @@ -318,7 +318,7 @@ bool haveRDRAND() noexcept inline bool haveAMX() noexcept { -#if defined(__x86_64__) || defined(__i386__) +#if defined(__x86_64__) // http://www.intel.com/content/dam/www/public/us/en/documents/manuals/64-ia-32-architectures-optimization-manual.pdf return haveOSXSAVE() // implies haveXSAVE() && ((our_xgetbv(0) >> 17) & 0x3) == 0x3; // AMX state are enabled by OS From 2f315e0eb54f1b9d93b6c5406287d83f748530ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 02:14:15 +0100 Subject: [PATCH 060/105] Style check for abbreviations --- src/Client/ConnectionParameters.cpp | 2 +- src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp | 2 +- src/Functions/serverConstants.cpp | 10 +++++----- src/IO/ConnectionTimeouts.cpp | 6 +++--- src/IO/ConnectionTimeouts.h | 4 ++-- src/IO/S3/Credentials.cpp | 5 +---- src/IO/S3/PocoHTTPClient.cpp | 4 ++-- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterShowFunctionsQuery.cpp | 4 ++-- src/Interpreters/Session.cpp | 4 ++-- src/Interpreters/Session.h | 2 +- .../Kusto/KustoFunctions/KQLFunctionFactory.cpp | 4 ++-- .../Kusto/KustoFunctions/KQLStringFunctions.cpp | 4 ++-- src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h | 4 ++-- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 4 ++-- src/Server/HTTPHandler.cpp | 8 ++++---- src/Storages/StorageURL.cpp | 2 +- utils/check-style/check-style | 6 ++++++ 19 files changed, 41 insertions(+), 38 deletions(-) diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index a0ae47f0620..5c22b6c6d3f 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -125,7 +125,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)) .withReceiveTimeout( Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)) - .withTcpKeepAliveTimeout( + .withTCPKeepAliveTimeout( Poco::Timespan(config.getInt("tcp_keep_alive_timeout", DEFAULT_TCP_KEEP_ALIVE_TIMEOUT), 0)) .withHandshakeTimeout( Poco::Timespan(config.getInt("handshake_timeout_ms", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000) * 1000)) diff --git a/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp b/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp index f1591943a12..b35e507b242 100644 --- a/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp +++ b/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp @@ -227,7 +227,7 @@ void parseMatchNode(UInt64 parent_id, UInt64 & id, const YAML::Node & node, Resu if (!match.contains(key_name)) { - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Yaml match rule must contain key {}", key_name); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "YAML match rule must contain key {}", key_name); } for (const auto & [key, node_] : match) { diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 9f1a3584df8..fd8fb22455b 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -51,12 +51,12 @@ namespace }; - class FunctionTcpPort : public FunctionConstantBase + class FunctionTCPPort : public FunctionConstantBase { public: static constexpr auto name = "tcpPort"; - static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTcpPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionTCPPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {} }; @@ -153,9 +153,9 @@ REGISTER_FUNCTION(ServerUUID) factory.registerFunction(); } -REGISTER_FUNCTION(TcpPort) +REGISTER_FUNCTION(TCPPort) { - factory.registerFunction(); + factory.registerFunction(); } REGISTER_FUNCTION(Timezone) diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index ecc0d64580b..f2db3169400 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -20,7 +20,7 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Setti .withConnectionTimeout(settings.connect_timeout) .withSendTimeout(settings.send_timeout) .withReceiveTimeout(settings.receive_timeout) - .withTcpKeepAliveTimeout(settings.tcp_keep_alive_timeout) + .withTCPKeepAliveTimeout(settings.tcp_keep_alive_timeout) .withHandshakeTimeout(settings.handshake_timeout_ms) .withHedgedConnectionTimeout(settings.hedged_connection_timeout_ms) .withReceiveDataTimeout(settings.receive_data_timeout_ms); @@ -40,8 +40,8 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings .withConnectionTimeout(settings.http_connection_timeout) .withSendTimeout(settings.http_send_timeout) .withReceiveTimeout(settings.http_receive_timeout) - .withHttpKeepAliveTimeout(http_keep_alive_timeout) - .withTcpKeepAliveTimeout(settings.tcp_keep_alive_timeout) + .withHTTPKeepAliveTimeout(http_keep_alive_timeout) + .withTCPKeepAliveTimeout(settings.tcp_keep_alive_timeout) .withHandshakeTimeout(settings.handshake_timeout_ms); } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 6967af08204..7fe97b5ec36 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -16,8 +16,8 @@ struct Settings; M(secure_connection_timeout, withSecureConnectionTimeout) \ M(send_timeout, withSendTimeout) \ M(receive_timeout, withReceiveTimeout) \ - M(tcp_keep_alive_timeout, withTcpKeepAliveTimeout) \ - M(http_keep_alive_timeout, withHttpKeepAliveTimeout) \ + M(tcp_keep_alive_timeout, withTCPKeepAliveTimeout) \ + M(http_keep_alive_timeout, withHTTPKeepAliveTimeout) \ M(hedged_connection_timeout, withHedgedConnectionTimeout) \ M(receive_data_timeout, withReceiveDataTimeout) \ M(handshake_timeout, withHandshakeTimeout) \ diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index e64f54b99ad..80366510b53 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes # include # include -# include # include # include @@ -31,9 +30,7 @@ namespace ErrorCodes # include # include -# include -# include # include # include # include @@ -755,7 +752,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.put_request_throttler, Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); - /// See MakeDefaultHttpResourceClientConfiguration(). + /// See MakeDefaultHTTPResourceClientConfiguration(). /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside /// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp aws_client_configuration.maxConnections = 2; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 21acdfd69f2..dbb93e63143 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -146,9 +146,9 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio .withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000)) .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) - .withTcpKeepAliveTimeout(Poco::Timespan( + .withTCPKeepAliveTimeout(Poco::Timespan( client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)) - .withHttpKeepAliveTimeout(Poco::Timespan( + .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 } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fdd18c9bdeb..1dbf54e675b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4513,7 +4513,7 @@ void Context::setClientConnectionId(uint32_t connection_id_) client_info.connection_id = connection_id_; } -void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Context::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) { client_info.http_method = http_method; client_info.http_user_agent = http_user_agent; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index cdd188faa48..a7908d45a9b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -630,7 +630,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); void setForwardedFor(const String & forwarded_for); void setQueryKind(ClientInfo::QueryKind query_kind); void setQueryKindInitial(); diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index e83f61eac53..829670d7929 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -25,13 +25,13 @@ String InterpreterShowFunctionsQuery::getRewrittenQuery() const auto & query = query_ptr->as(); - DatabasePtr systemDb = DatabaseCatalog::instance().getSystemDatabase(); + DatabasePtr system_db = DatabaseCatalog::instance().getSystemDatabase(); String rewritten_query = fmt::format( R"( SELECT * FROM {}.{})", - systemDb->getDatabaseName(), + system_db->getDatabaseName(), functions_table); if (!query.like.empty()) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index df97a09f686..b52f8a507e3 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -429,11 +429,11 @@ void Session::setClientConnectionId(uint32_t connection_id) prepared_client_info->connection_id = connection_id; } -void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Session::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) { if (session_context) { - session_context->setHttpClientInfo(http_method, http_user_agent, http_referer); + session_context->setHTTPClientInfo(http_method, http_user_agent, http_referer); } else { diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index cde000d89fa..334560a33c8 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -65,7 +65,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); void setForwardedFor(const String & forwarded_for); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); diff --git a/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp index adac892b49d..044cc2e0622 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp +++ b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp @@ -359,7 +359,7 @@ std::unique_ptr KQLFunctionFactory::get(String & kql_functio return std::make_unique(); case KQLFunctionValue::extract_json: - return std::make_unique(); + return std::make_unique(); case KQLFunctionValue::has_any_index: return std::make_unique(); @@ -389,7 +389,7 @@ std::unique_ptr KQLFunctionFactory::get(String & kql_functio return std::make_unique(); case KQLFunctionValue::parse_json: - return std::make_unique(); + return std::make_unique(); case KQLFunctionValue::parse_url: return std::make_unique(); diff --git a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp index 0f9ca67d6dc..afb8809c69e 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp +++ b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp @@ -240,7 +240,7 @@ bool ExtractAll::convertImpl(String & out, IParser::Pos & pos) return true; } -bool ExtractJson::convertImpl(String & out, IParser::Pos & pos) +bool ExtractJSON::convertImpl(String & out, IParser::Pos & pos) { String datatype = "String"; ParserKeyword s_kql("typeof"); @@ -431,7 +431,7 @@ bool ParseCSV::convertImpl(String & out, IParser::Pos & pos) return true; } -bool ParseJson::convertImpl(String & out, IParser::Pos & pos) +bool ParseJSON::convertImpl(String & out, IParser::Pos & pos) { const String fn_name = getKQLFunctionName(pos); if (fn_name.empty()) diff --git a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h index 492a59263ec..e55d07defd0 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h +++ b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h @@ -62,7 +62,7 @@ protected: bool convertImpl(String & out, IParser::Pos & pos) override; }; -class ExtractJson : public IParserKQLFunction +class ExtractJSON : public IParserKQLFunction { protected: const char * getName() const override { return "extract_json(), extractjson()"; } @@ -125,7 +125,7 @@ protected: bool convertImpl(String & out, IParser::Pos & pos) override; }; -class ParseJson : public IParserKQLFunction +class ParseJJSON : public IParserKQLFunction { protected: const char * getName() const override { return "parse_json()"; } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 8dc8fa516dc..8ef2cda5587 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -212,7 +212,7 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No }; } -static std::string nodeToJson(avro::NodePtr root_node) +static std::string nodeToJSON(avro::NodePtr root_node) { std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss.exceptions(std::ios::failbit); @@ -641,7 +641,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Type {} is not compatible with Avro {}:\n{}", - target_type->getName(), avro::toString(root_node->type()), nodeToJson(root_node)); + target_type->getName(), avro::toString(root_node->type()), nodeToJSON(root_node)); } AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(const avro::NodePtr & root_node) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 72e7c5552f8..35a95c0534d 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -125,7 +125,7 @@ namespace ErrorCodes namespace { -bool tryAddHttpOptionHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) +bool tryAddHTTPOptionHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) { if (config.has("http_options_response")) { @@ -153,7 +153,7 @@ bool tryAddHttpOptionHeadersFromConfig(HTTPServerResponse & response, const Poco void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) { /// If can add some headers from config - if (tryAddHttpOptionHeadersFromConfig(response, config)) + if (tryAddHTTPOptionHeadersFromConfig(response, config)) { response.setKeepAlive(false); response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT); @@ -496,7 +496,7 @@ bool HTTPHandler::authenticateUser( else if (request.getMethod() == HTTPServerRequest::HTTP_POST) http_method = ClientInfo::HTTPMethod::POST; - session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); + session->setHTTPClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); session->setForwardedFor(request.get("X-Forwarded-For", "")); session->setQuotaClientKey(quota_key); @@ -1065,7 +1065,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse response.set("X-ClickHouse-Server-Display-Name", server_display_name); if (!request.get("Origin", "").empty()) - tryAddHttpOptionHeadersFromConfig(response, server.config()); + tryAddHTTPOptionHeadersFromConfig(response, server.config()); /// For keep-alive to work. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 547342cf439..6f3599630d3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1511,7 +1511,7 @@ void StorageURL::processNamedCollectionResult(Configuration & configuration, con && configuration.http_method != Poco::Net::HTTPRequest::HTTP_PUT) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Http method can be POST or PUT (current: {}). For insert default is POST, for select GET", + "HTTP method can be POST or PUT (current: {}). For insert default is POST, for select GET", configuration.http_method); configuration.format = collection.getOrDefault("format", "auto"); diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f0745ab43f3..6c12970c4bb 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -442,3 +442,9 @@ ls -1d $ROOT_PATH/contrib/*-cmake | xargs -I@ find @ -name 'CMakeLists.txt' -or # DOS/Windows newlines find $ROOT_PATH/{base,src,programs,utils,docs} -name '*.md' -or -name '*.h' -or -name '*.cpp' -or -name '*.js' -or -name '*.py' -or -name '*.html' | xargs grep -l -P '\r$' && echo "^ Files contain DOS/Windows newlines (\r\n instead of \n)." + +# Wrong spelling of abbreviations, e.g. SQL is right, Sql is wrong. XMLHttpRequest is very wrong. +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | + grep -vP $EXCLUDE_DIRS | + xargs grep -P 'Sql|Html|Xml|Cpu|Tcp|Udp|Http|Db|Json|Yaml' | grep -v -P 'RabbitMQ|Azure|Aws|aws|Avro|IO/S3' && + echo "Abbreviations such as SQL, XML, HTTP, should be in all caps. For example, SQL is right, Sql is wrong. XMLHttpRequest is very wrong." From 9686bb51bcbd0e1272ae8f8ad0b5265aff71a6c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 02:15:14 +0100 Subject: [PATCH 061/105] Style check for abbreviations --- src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h index e55d07defd0..9b0c6327e01 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h +++ b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h @@ -125,7 +125,7 @@ protected: bool convertImpl(String & out, IParser::Pos & pos) override; }; -class ParseJJSON : public IParserKQLFunction +class ParseJSON : public IParserKQLFunction { protected: const char * getName() const override { return "parse_json()"; } From d10b18afeb121b47b7ccef384550e9eb765391e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 02:21:55 +0100 Subject: [PATCH 062/105] I found TODO in the code, that can be implemented --- src/Client/QueryFuzzer.cpp | 24 +++++++----------------- 1 file changed, 7 insertions(+), 17 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 24be7491ec7..0597a7c1eed 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1168,23 +1168,13 @@ void QueryFuzzer::fuzz(ASTPtr & ast) fuzz(select->children); } - /* - * The time to fuzz the settings has not yet come. - * Apparently we don't have any infrastructure to validate the values of - * the settings, and the first query with max_block_size = -1 breaks - * because of overflows here and there. - *//* - * else if (auto * set = typeid_cast(ast.get())) - * { - * for (auto & c : set->changes) - * { - * if (fuzz_rand() % 50 == 0) - * { - * c.value = fuzzField(c.value); - * } - * } - * } - */ + else if (auto * set = typeid_cast(ast.get())) + { + /// Fuzz settings + for (auto & c : set->changes) + if (fuzz_rand() % 50 == 0) + c.value = fuzzField(c.value); + } else if (auto * literal = typeid_cast(ast.get())) { // There is a caveat with fuzzing the children: many ASTs also keep the From 3128cf1f1b2ea4448822f2a3f5c7825d49258641 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 02:24:21 +0100 Subject: [PATCH 063/105] Remove the check for formatting consistency from the Fuzzer --- programs/client/Client.cpp | 78 +------------------------------------- 1 file changed, 1 insertion(+), 77 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index fdd262f185d..7a77b7dd0ec 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -845,83 +845,7 @@ bool Client::processWithFuzzing(const String & full_query) have_error = true; } - // Check that after the query is formatted, we can parse it back, - // format again and get the same result. Unfortunately, we can't - // compare the ASTs, which would be more sensitive to errors. This - // double formatting check doesn't catch all errors, e.g. we can - // format query incorrectly, but to a valid SQL that we can then - // parse and format into the same SQL. - // There are some complicated cases where we can generate the SQL - // which we can't parse: - // * first argument of lambda() replaced by fuzzer with - // something else, leading to constructs such as - // arrayMap((min(x) + 3) -> x + 1, ....) - // * internals of Enum replaced, leading to: - // Enum(equals(someFunction(y), 3)). - // And there are even the cases when we can parse the query, but - // it's logically incorrect and its formatting is a mess, such as - // when `lambda()` function gets substituted into a wrong place. - // To avoid dealing with these cases, run the check only for the - // queries we were able to successfully execute. - // Another caveat is that sometimes WITH queries are not executed, - // if they are not referenced by the main SELECT, so they can still - // have the aforementioned problems. Disable this check for such - // queries, for lack of a better solution. - // There is also a problem that fuzzer substitutes positive Int64 - // literals or Decimal literals, which are then parsed back as - // UInt64, and suddenly duplicate alias substitution starts or stops - // working (ASTWithAlias::formatImpl) or something like that. - // So we compare not even the first and second formatting of the - // query, but second and third. - // If you have to add any more workarounds to this check, just remove - // it altogether, it's not so useful. - if (ast_to_process && !have_error && !queryHasWithClause(*ast_to_process)) - { - ASTPtr ast_2; - try - { - const auto * tmp_pos = query_to_execute.c_str(); - ast_2 = parseQuery(tmp_pos, tmp_pos + query_to_execute.size(), false /* allow_multi_statements */); - } - catch (Exception & e) - { - if (e.code() != ErrorCodes::SYNTAX_ERROR && - e.code() != ErrorCodes::TOO_DEEP_RECURSION) - throw; - } - - if (ast_2) - { - const auto text_2 = ast_2->formatForErrorMessage(); - const auto * tmp_pos = text_2.c_str(); - const auto ast_3 = parseQuery(tmp_pos, tmp_pos + text_2.size(), - false /* allow_multi_statements */); - const auto text_3 = ast_3 ? ast_3->formatForErrorMessage() : ""; - - if (text_3 != text_2) - { - fmt::print(stderr, "Found error: The query formatting is broken.\n"); - - printChangedSettings(); - - fmt::print(stderr, - "Got the following (different) text after formatting the fuzzed query and parsing it back:\n'{}'\n, expected:\n'{}'\n", - text_3, text_2); - fmt::print(stderr, "In more detail:\n"); - fmt::print(stderr, "AST-1 (generated by fuzzer):\n'{}'\n", ast_to_process->dumpTree()); - fmt::print(stderr, "Text-1 (AST-1 formatted):\n'{}'\n", query_to_execute); - fmt::print(stderr, "AST-2 (Text-1 parsed):\n'{}'\n", ast_2->dumpTree()); - fmt::print(stderr, "Text-2 (AST-2 formatted):\n'{}'\n", text_2); - fmt::print(stderr, "AST-3 (Text-2 parsed):\n'{}'\n", ast_3 ? ast_3->dumpTree() : ""); - fmt::print(stderr, "Text-3 (AST-3 formatted):\n'{}'\n", text_3); - fmt::print(stderr, "Text-3 must be equal to Text-2, but it is not.\n"); - - _exit(1); - } - } - } - - // The server is still alive so we're going to continue fuzzing. + // The server is still alive, so we're going to continue fuzzing. // Determine what we're going to use as the starting AST. if (have_error) { From 52bcecf7b02cfff242164d2158ef10af342685f9 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Fri, 16 Feb 2024 11:38:28 -0800 Subject: [PATCH 064/105] Bump curl to version 4.6.0 --- contrib/curl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/curl b/contrib/curl index 7161cb17c01..5ce164e0e92 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit 7161cb17c01dcff1dc5bf89a18437d9d729f1ecd +Subproject commit 5ce164e0e9290c96eb7d502173426c0a135ec008 From 20b9e519ffd30108e66a4457240a59696bca0181 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 04:59:32 +0100 Subject: [PATCH 065/105] Add a test --- .../1_stateful/00098_primary_key_memory_allocated.reference | 1 + tests/queries/1_stateful/00098_primary_key_memory_allocated.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/1_stateful/00098_primary_key_memory_allocated.reference create mode 100644 tests/queries/1_stateful/00098_primary_key_memory_allocated.sql diff --git a/tests/queries/1_stateful/00098_primary_key_memory_allocated.reference b/tests/queries/1_stateful/00098_primary_key_memory_allocated.reference new file mode 100644 index 00000000000..72749c905a3 --- /dev/null +++ b/tests/queries/1_stateful/00098_primary_key_memory_allocated.reference @@ -0,0 +1 @@ +1 1 1 diff --git a/tests/queries/1_stateful/00098_primary_key_memory_allocated.sql b/tests/queries/1_stateful/00098_primary_key_memory_allocated.sql new file mode 100644 index 00000000000..7371678a0f6 --- /dev/null +++ b/tests/queries/1_stateful/00098_primary_key_memory_allocated.sql @@ -0,0 +1 @@ +SELECT primary_key_bytes_in_memory < 16000, primary_key_bytes_in_memory_allocated < 16000, primary_key_bytes_in_memory_allocated / primary_key_bytes_in_memory < 1.1 FROM system.parts WHERE database = 'test' AND table = 'hits'; From b5efbe62e7dc25f6049571348ccf18d34ba5a191 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 05:41:52 +0100 Subject: [PATCH 066/105] Avoid overflow in settings --- src/Core/SettingsFields.cpp | 7 ++++--- src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 2 -- .../queries/0_stateless/02992_settings_overflow.reference | 0 tests/queries/0_stateless/02992_settings_overflow.sql | 1 + 4 files changed, 5 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02992_settings_overflow.reference create mode 100644 tests/queries/0_stateless/02992_settings_overflow.sql diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 80197cfbe22..a5258b3d0b3 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -2,10 +2,11 @@ #include #include -#include +#include #include #include #include +#include #include #include #include @@ -50,7 +51,7 @@ namespace if (f.getType() == Field::Types::String) return stringToNumber(f.get()); else - return applyVisitor(FieldVisitorConvertToNumber(), f); + return static_cast(convertFieldToTypeOrThrow(f, DataTypeNumber>()).template get()); } Map stringToMap(const String & str) @@ -174,7 +175,7 @@ namespace if (f.getType() == Field::Types::String) return stringToMaxThreads(f.get()); else - return applyVisitor(FieldVisitorConvertToNumber(), f); + return convertFieldToTypeOrThrow(f, DataTypeUInt64()).template get(); } } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index bf584b759f8..f91f7cf536b 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -28,7 +27,6 @@ #include #include #include -#include namespace DB diff --git a/tests/queries/0_stateless/02992_settings_overflow.reference b/tests/queries/0_stateless/02992_settings_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02992_settings_overflow.sql b/tests/queries/0_stateless/02992_settings_overflow.sql new file mode 100644 index 00000000000..cbe14788709 --- /dev/null +++ b/tests/queries/0_stateless/02992_settings_overflow.sql @@ -0,0 +1 @@ +SET max_threads = -1; -- { serverError ARGUMENT_OUT_OF_BOUND } From 96113a46c5f0f6d5488917e6657ce89330d2ee7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 05:52:24 +0100 Subject: [PATCH 067/105] Update LZ4 to the latest dev --- contrib/lz4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/lz4 b/contrib/lz4 index 92ebf1870b9..ce45a9dbdb0 160000 --- a/contrib/lz4 +++ b/contrib/lz4 @@ -1 +1 @@ -Subproject commit 92ebf1870b9acbefc0e7970409a181954a10ff40 +Subproject commit ce45a9dbdb059511a3e9576b19db3e7f1a4f172e From ac5bef7c74f98fdc8423d8ef5c9d78f1351d79cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:16:01 +0100 Subject: [PATCH 068/105] A small preparation for better handling of primary key in memory --- src/Processors/QueryPlan/PartsSplitter.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 21 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- 7 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 363fdca22c5..0fc6ddd6408 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -54,7 +54,7 @@ public: Values getValue(size_t part_idx, size_t mark) const { - const auto & index = parts[part_idx].data_part->index; + const auto & index = parts[part_idx].data_part->getIndex(); Values values(index.size()); for (size_t i = 0; i < values.size(); ++i) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0f82e00edff..bc64632356f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -313,13 +313,13 @@ IMergeTreeDataPart::IMergeTreeDataPart( const IMergeTreeDataPart * parent_part_) : DataPartStorageHolder(data_part_storage_) , storage(storage_) - , mutable_name(name_) , name(mutable_name) , info(info_) , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) , parent_part_name(parent_part ? parent_part->name : "") + , mutable_name(name_) { if (parent_part) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index fcf9d5bd17d..878258bddf0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -75,6 +75,7 @@ public: using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; + using Index = Columns; using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; @@ -212,10 +213,6 @@ public: const MergeTreeData & storage; -private: - String mutable_name; - mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary}; - public: const String & name; // const ref to private mutable_name MergeTreePartInfo info; @@ -309,12 +306,6 @@ public: /// Throws an exception if state of the part is not in affordable_states void assertState(const std::initializer_list & affordable_states) const; - /// Primary key (correspond to primary.idx file). - /// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple. - /// Note that marks (also correspond to primary key) is not always in RAM, but cached. See MarkCache.h. - using Index = Columns; - Index index; - MergeTreePartition partition; /// Amount of rows between marks @@ -369,6 +360,9 @@ public: /// Version of part metadata (columns, pk and so on). Managed properly only for replicated merge tree. int32_t metadata_version; + const Index & getIndex() const { return index; } + void setIndex(Columns index_) { index = std::move(index_); } + /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; @@ -567,6 +561,10 @@ public: mutable std::atomic last_removal_attempt_time = 0; protected: + /// Primary key (correspond to primary.idx file). + /// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple. + /// Note that marks (also correspond to primary key) is not always in RAM, but cached. See MarkCache.h. + Index index; /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; @@ -623,6 +621,9 @@ protected: void initializeIndexGranularityInfo(); private: + String mutable_name; + mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary}; + /// In compact parts order of columns is necessary NameToNumber column_name_to_position; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c8262914702..e14a358745e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6824,7 +6824,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( { for (const auto & part : real_parts) { - const auto & primary_key_column = *part->index[0]; + const auto & primary_key_column = *part->getIndex()[0]; auto & min_column = assert_cast(*partition_minmax_count_columns[pos]); insert(min_column, primary_key_column[0]); } @@ -6835,7 +6835,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( { for (const auto & part : real_parts) { - const auto & primary_key_column = *part->index[0]; + const auto & primary_key_column = *part->getIndex()[0]; auto & max_column = assert_cast(*partition_minmax_count_columns[pos]); insert(max_column, primary_key_column[primary_key_column.size() - 1]); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a76d370d057..1ba28713680 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1087,7 +1087,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( MarkRanges res; size_t marks_count = part->index_granularity.getMarksCount(); - const auto & index = part->index; + const auto & index = part->getIndex(); if (marks_count == 0) return res; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 1d10a1433ef..f2fe2e0f255 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -181,7 +181,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); - new_part->index = writer->releaseIndexColumns(); + new_part->setIndex(writer->releaseIndexColumns()); new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk()); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9959688d889..6882963fd24 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -899,7 +899,7 @@ void finalizeMutatedPart( new_data_part->rows_count = source_part->rows_count; new_data_part->index_granularity = source_part->index_granularity; - new_data_part->index = source_part->index; + new_data_part->setIndex(source_part->getIndex()); new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); From fce3a8cafc13b4b454ed53966b9f7fd1d7f845fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:16:56 +0100 Subject: [PATCH 069/105] A small preparation for better handling of primary key in memory --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 878258bddf0..e82dc8fc2a3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -563,7 +563,7 @@ public: protected: /// Primary key (correspond to primary.idx file). /// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple. - /// Note that marks (also correspond to primary key) is not always in RAM, but cached. See MarkCache.h. + /// Note that marks (also correspond to primary key) are not always in RAM, but cached. See MarkCache.h. Index index; /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk From e9cf92245f126d8b92cd29747f38bf94e0b466c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:20:44 +0100 Subject: [PATCH 070/105] Move methods to .cpp --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 +++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 ++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index bc64632356f..70a8cee6106 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -342,6 +342,19 @@ IMergeTreeDataPart::~IMergeTreeDataPart() decrementTypeMetric(part_type); } + +const IMergeTreeDataPart::Index & IMergeTreeDataPart::getIndex() const +{ + return index; +} + + +void IMergeTreeDataPart::setIndex(Columns index_) +{ + index = std::move(index_); +} + + void IMergeTreeDataPart::setName(const String & new_name) { mutable_name = new_name; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index e82dc8fc2a3..f93b1a4c9c1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -360,8 +360,8 @@ public: /// Version of part metadata (columns, pk and so on). Managed properly only for replicated merge tree. int32_t metadata_version; - const Index & getIndex() const { return index; } - void setIndex(Columns index_) { index = std::move(index_); } + const Index & getIndex() const; + void setIndex(Columns index_); /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; From dc0b3e8bb5c0b1de5b7e2dd7a4b02e22ee800d35 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:33:13 +0100 Subject: [PATCH 071/105] Update test --- .../0_stateless/01557_field_infinite_convert_to_number.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql index edc4d5cbc91..2ceea259de0 100644 --- a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql +++ b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql @@ -1 +1 @@ -SET max_threads = nan; -- { serverError 70 } +SET max_threads = nan; -- { serverError 69 } From 423ff759f1ee62bde0000b441751b3fdad24fdc0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:35:03 +0100 Subject: [PATCH 072/105] Lazy loading of primary keys in memory --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 11 +++++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 8 +++++--- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 70a8cee6106..240010a7550 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -345,13 +345,21 @@ IMergeTreeDataPart::~IMergeTreeDataPart() const IMergeTreeDataPart::Index & IMergeTreeDataPart::getIndex() const { + std::scoped_lock lock(index_mutex); + if (!index_loaded) + loadIndex(lock); + index_loaded = true; return index; } void IMergeTreeDataPart::setIndex(Columns index_) { + std::scoped_lock lock(index_mutex); + if (!index.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The index of data part can be set only once"); index = std::move(index_); + index_loaded = true; } @@ -683,7 +691,6 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadChecksums(require_columns_checksums); loadIndexGranularity(); calculateColumnsAndSecondaryIndicesSizesOnDisk(); - loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`. loadPartitionAndMinMaxIndex(); bool has_broken_projections = false; @@ -817,7 +824,7 @@ void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) co { } -void IMergeTreeDataPart::loadIndex() +void IMergeTreeDataPart::loadIndex(std::scoped_lock &) const { /// It can be empty in case of mutations if (!index_granularity.isInitialized()) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f93b1a4c9c1..0fc874b7953 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -562,9 +562,11 @@ public: protected: /// Primary key (correspond to primary.idx file). - /// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple. + /// Lazily loaded in RAM. Contains each index_granularity-th value of primary key tuple. /// Note that marks (also correspond to primary key) are not always in RAM, but cached. See MarkCache.h. - Index index; + mutable std::mutex index_mutex; + mutable Index index; + mutable bool index_loaded = false; /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; @@ -662,7 +664,7 @@ private: virtual void appendFilesOfIndexGranularity(Strings & files) const; /// Loads index file. - void loadIndex(); + void loadIndex(std::scoped_lock &) const; void appendFilesOfIndex(Strings & files) const; From 021e11e4fc674fa8eb5ea7fec17563dbfea58a22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 06:42:53 +0100 Subject: [PATCH 073/105] Add a setting for lazy loading --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + 2 files changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 240010a7550..fee7286da3b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -690,6 +690,10 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadColumns(require_columns_checksums); loadChecksums(require_columns_checksums); loadIndexGranularity(); + + if (!storage.getSettings()->primary_key_lazy_load) + getIndex(); + calculateColumnsAndSecondaryIndicesSizesOnDisk(); loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`. loadPartitionAndMinMaxIndex(); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 96cab9c0293..b64632b6139 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -201,6 +201,7 @@ struct Settings; M(String, primary_key_compression_codec, "ZSTD(3)", "Compression encoding used by primary, primary key is small enough and cached, so the default compression is ZSTD(3).", 0) \ M(UInt64, marks_compress_block_size, 65536, "Mark compress block size, the actual size of the block to compress.", 0) \ M(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ + M(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ From 9acd87aa8c847fe00db1be879f4bd83b9d9aea1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 07:36:47 +0100 Subject: [PATCH 074/105] Add a test --- .../02993_lazy_index_loading.reference | 4 ++++ .../0_stateless/02993_lazy_index_loading.sql | 18 ++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02993_lazy_index_loading.reference create mode 100644 tests/queries/0_stateless/02993_lazy_index_loading.sql diff --git a/tests/queries/0_stateless/02993_lazy_index_loading.reference b/tests/queries/0_stateless/02993_lazy_index_loading.reference new file mode 100644 index 00000000000..5313c84136e --- /dev/null +++ b/tests/queries/0_stateless/02993_lazy_index_loading.reference @@ -0,0 +1,4 @@ +100000000 140000000 +0 0 +1 +100000000 140000000 diff --git a/tests/queries/0_stateless/02993_lazy_index_loading.sql b/tests/queries/0_stateless/02993_lazy_index_loading.sql new file mode 100644 index 00000000000..7de4af9ef0e --- /dev/null +++ b/tests/queries/0_stateless/02993_lazy_index_loading.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; + +INSERT INTO test SELECT randomString(1000) FROM numbers(100000); +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; + +DETACH TABLE test; +SET max_memory_usage = '50M'; +ATTACH TABLE test; + +SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table = 'test'; + +SET max_memory_usage = '200M'; +SELECT s != '' FROM test LIMIT 1; + +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; + +DROP TABLE test; From 3d21d546a92a15e77470648a9652f2f62a4d6825 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 09:44:10 +0100 Subject: [PATCH 075/105] Better code --- src/Common/FieldVisitorConvertToNumber.h | 2 +- src/Core/AccurateComparison.h | 2 +- src/Core/SettingsFields.cpp | 49 +++++++++++++++++++++--- 3 files changed, 46 insertions(+), 7 deletions(-) diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index bf8c8c8638e..47a1e669969 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -91,7 +91,7 @@ public: if constexpr (std::is_floating_point_v) return x.getValue().template convertTo() / x.getScaleMultiplier().template convertTo(); else - return (x.getValue() / x.getScaleMultiplier()). template convertTo(); + return (x.getValue() / x.getScaleMultiplier()).template convertTo(); } T operator() (const AggregateFunctionStateData &) const diff --git a/src/Core/AccurateComparison.h b/src/Core/AccurateComparison.h index a201c136e3a..139ee4d88dc 100644 --- a/src/Core/AccurateComparison.h +++ b/src/Core/AccurateComparison.h @@ -152,7 +152,7 @@ bool notEqualsOp(A a, B b) } /// Converts numeric to an equal numeric of other type. -/// When `strict` is `true` check that result exactly same as input, otherwise just check overflow +/// When `strict` is `true` check that result exactly the same as input, otherwise just check overflow template inline bool NO_SANITIZE_UNDEFINED convertNumeric(From value, To & result) { diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index a5258b3d0b3..223bdc1aeb7 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -1,12 +1,11 @@ #include - #include +#include #include -#include +#include #include #include #include -#include #include #include #include @@ -14,6 +13,7 @@ #include + namespace DB { namespace ErrorCodes @@ -21,6 +21,7 @@ namespace ErrorCodes extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int CANNOT_PARSE_BOOL; extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_CONVERT_TYPE; } @@ -49,9 +50,47 @@ namespace T fieldToNumber(const Field & f) { if (f.getType() == Field::Types::String) + { return stringToNumber(f.get()); + } + else if (f.getType() == Field::Types::UInt64) + { + T result; + if (!accurate::convertNumeric(f.get(), result)) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); + return result; + } + else if (f.getType() == Field::Types::Int64) + { + T result; + if (!accurate::convertNumeric(f.get(), result)) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); + return result; + } + else if (f.getType() == Field::Types::Float64) + { + Float64 x = f.get(); + if constexpr (std::is_floating_point_v) + { + return T(x); + } + else + { + if (!isFinite(x)) + { + /// Conversion of infinite values to integer is undefined. + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert infinite value to integer type"); + } + else if (x > Float64(std::numeric_limits::max()) || x < Float64(std::numeric_limits::lowest())) + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert out of range floating point value to integer type"); + } + else + return T(x); + } + } else - return static_cast(convertFieldToTypeOrThrow(f, DataTypeNumber>()).template get()); + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Invalid value {} of the setting, which needs {}", f, demangle(typeid(T).name())); } Map stringToMap(const String & str) @@ -175,7 +214,7 @@ namespace if (f.getType() == Field::Types::String) return stringToMaxThreads(f.get()); else - return convertFieldToTypeOrThrow(f, DataTypeUInt64()).template get(); + return fieldToNumber(f); } } From dbb0ed8157461eeafba98cc14e868c4536052687 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 10:08:36 +0100 Subject: [PATCH 076/105] Fix error --- src/Core/SettingsFields.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 223bdc1aeb7..9562f45a35d 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -67,6 +67,13 @@ namespace throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); return result; } + else if (f.getType() == Field::Types::Bool) + { + if constexpr (std::is_same_v) + return f.get(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion of a bool value {} to {} looks suspicious", f, demangle(typeid(T).name())); + } else if (f.getType() == Field::Types::Float64) { Float64 x = f.get(); From b7e563c0a699c416f8d38ac590c5a631aa2ea4a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 11:13:54 +0100 Subject: [PATCH 077/105] Update test --- tests/queries/0_stateless/02993_lazy_index_loading.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02993_lazy_index_loading.reference b/tests/queries/0_stateless/02993_lazy_index_loading.reference index 5313c84136e..5bc329ae4eb 100644 --- a/tests/queries/0_stateless/02993_lazy_index_loading.reference +++ b/tests/queries/0_stateless/02993_lazy_index_loading.reference @@ -1,4 +1,4 @@ 100000000 140000000 0 0 1 -100000000 140000000 +100000000 100000000 From f075961728c69ace187aec038e2a21ee38ce6751 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Feb 2024 11:37:31 +0100 Subject: [PATCH 078/105] Update --- src/Core/SettingsFields.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 9562f45a35d..d1a1439fb15 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -69,10 +69,7 @@ namespace } else if (f.getType() == Field::Types::Bool) { - if constexpr (std::is_same_v) - return f.get(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion of a bool value {} to {} looks suspicious", f, demangle(typeid(T).name())); + return T(f.get()); } else if (f.getType() == Field::Types::Float64) { From 3323d5ce81914d93026f2cf68b68170cf8c4053b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 17 Feb 2024 15:02:26 +0100 Subject: [PATCH 079/105] Move threadPoolCallbackRunner to the "Common" folder. --- src/Backups/BackupIO_AzureBlobStorage.cpp | 2 +- src/Backups/BackupIO_S3.cpp | 2 +- src/{Interpreters => Common}/threadPoolCallbackRunner.h | 0 src/Disks/IO/ThreadPoolReader.h | 2 +- src/Disks/IO/ThreadPoolRemoteFSReader.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 2 +- src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h | 2 +- src/IO/ParallelReadBuffer.h | 2 +- src/IO/S3/copyS3File.h | 2 +- src/IO/WriteBufferFromS3.h | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 2 +- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/System/StorageSystemDetachedParts.cpp | 2 +- 19 files changed, 18 insertions(+), 18 deletions(-) rename src/{Interpreters => Common}/threadPoolCallbackRunner.h (100%) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 52ce20d5108..b3b92323109 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -2,7 +2,7 @@ #if USE_AZURE_BLOB_STORAGE #include -#include +#include #include #include #include diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index fa4c1af3698..f8bbf5b1f79 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -2,7 +2,7 @@ #if USE_AWS_S3 #include -#include +#include #include #include #include diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Common/threadPoolCallbackRunner.h similarity index 100% rename from src/Interpreters/threadPoolCallbackRunner.h rename to src/Common/threadPoolCallbackRunner.h diff --git a/src/Disks/IO/ThreadPoolReader.h b/src/Disks/IO/ThreadPoolReader.h index 42bc9bf8bb4..b8aff9f22a2 100644 --- a/src/Disks/IO/ThreadPoolReader.h +++ b/src/Disks/IO/ThreadPoolReader.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h index cd2bf223f33..abc251b2b10 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.h +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 049935ad60c..56c269a3fc5 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -22,7 +22,7 @@ #include #include #include -#include +#include #include #include "config.h" diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index a6abe03bac9..5f63e5f6e8a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 4cc49288af6..5771eb1ebe0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 83814f42693..1433f8d18ba 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index e76b40f77b7..daac1190399 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 607be51ed25..093d26ba7bb 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -5,7 +5,7 @@ #if USE_AWS_S3 #include -#include +#include #include #include #include diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 230f39b074e..5dc269990a1 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4ad440dae00..1de79ed17ca 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 7531c03a011..aeb6afeff11 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 47c2fe07bb4..8d8b0f1cc79 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index a450505f7a8..e1d1d0951e4 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 73559ef8571..587145cd1a7 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6f3599630d3..608e44c3cd0 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index a9cd5f2610a..3dae43976f7 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include From dfc4b72ab2d596405cef922eee1323edaea2e262 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 02:10:05 +0100 Subject: [PATCH 080/105] Update tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- .../0_stateless/01557_field_infinite_convert_to_number.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql index 2ceea259de0..cc71c8e6f6c 100644 --- a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql +++ b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql @@ -1 +1 @@ -SET max_threads = nan; -- { serverError 69 } +SET max_threads = nan; -- { serverError CANNOT_CONVERT_TYPE } From 2e95c689eb72f81a3cd078870c3dbda4ac038d3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 02:10:24 +0100 Subject: [PATCH 081/105] Update 02992_settings_overflow.sql --- tests/queries/0_stateless/02992_settings_overflow.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02992_settings_overflow.sql b/tests/queries/0_stateless/02992_settings_overflow.sql index cbe14788709..d120c3400e5 100644 --- a/tests/queries/0_stateless/02992_settings_overflow.sql +++ b/tests/queries/0_stateless/02992_settings_overflow.sql @@ -1 +1 @@ -SET max_threads = -1; -- { serverError ARGUMENT_OUT_OF_BOUND } +SET max_threads = -1; -- { serverError CANNOT_CONVERT_TYPE } From c50f8e79542eae88709cbe241bfb94710ac96af8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 02:12:31 +0100 Subject: [PATCH 082/105] Remove a header --- src/Core/SettingsFields.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index d1a1439fb15..f72b64fd56d 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include From c083498cf87b4181614228f2e796369bafb09bfe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 05:13:17 +0100 Subject: [PATCH 083/105] Apply review comments --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 +++ src/Storages/MergeTree/MergeTreeMarksLoader.cpp | 1 - 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b0eff01a4e7..2826c3e23f1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -830,6 +830,9 @@ void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) co void IMergeTreeDataPart::loadIndex(std::scoped_lock &) const { + /// Memory for index must not be accounted as memory usage for query, because it belongs to a table. + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + /// It can be empty in case of mutations if (!index_granularity.isInitialized()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index granularity is not loaded before index loading"); diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 7531c03a011..eb7b60b0727 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include From 232fe0dd1b62974e6bcef053b8a13cdc4694fd07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 05:38:34 +0100 Subject: [PATCH 084/105] Fix inconsistent formatting of SELECT ... FROM (EXPLAIN ...) --- src/Analyzer/QueryNode.cpp | 5 +---- src/Analyzer/UnionNode.cpp | 5 +---- src/Interpreters/TreeRewriter.cpp | 3 +-- src/Parsers/ASTSelectWithUnionQuery.cpp | 3 +-- src/Parsers/ASTSubquery.h | 7 +++++++ src/Parsers/ExpressionElementParsers.cpp | 7 +++---- src/Parsers/ExpressionListParsers.cpp | 6 ++---- src/Parsers/Kusto/ParserKQLQuery.cpp | 18 ++++++++---------- src/Storages/StorageView.cpp | 3 +-- src/TableFunctions/TableFunctionExplain.cpp | 20 ++++++++++++++++++-- 10 files changed, 43 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index d0bff759dea..bc7a29247e4 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -421,11 +421,8 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const if (is_subquery) { - auto subquery = std::make_shared(); - + auto subquery = std::make_shared(std::move(result_select_query)); subquery->cte_name = cte_name; - subquery->children.push_back(std::move(result_select_query)); - return subquery; } diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 5d2ac128abe..c6003126554 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -185,11 +185,8 @@ ASTPtr UnionNode::toASTImpl(const ConvertToASTOptions & options) const if (is_subquery) { - auto subquery = std::make_shared(); - + auto subquery = std::make_shared(std::move(select_with_union_query)); subquery->cte_name = cte_name; - subquery->children.push_back(std::move(select_with_union_query)); - return subquery; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ecd021328e7..bb6df2da8d9 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -262,8 +262,7 @@ struct ExistsExpressionData select_with_union_query->list_of_selects->children.push_back(std::move(select_query)); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - auto new_subquery = std::make_shared(); - new_subquery->children.push_back(select_with_union_query); + auto new_subquery = std::make_shared(std::move(select_with_union_query)); auto function = makeASTFunction("in", std::make_shared(1u), new_subquery); func = *function; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 48b4ae3c38d..c377e4bd66b 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -71,8 +71,7 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F } else { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); + auto sub_query = std::make_shared(*it); sub_query->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTSubquery.h b/src/Parsers/ASTSubquery.h index ef277a63126..e92a88b04dd 100644 --- a/src/Parsers/ASTSubquery.h +++ b/src/Parsers/ASTSubquery.h @@ -26,6 +26,13 @@ public: return clone; } + ASTSubquery() = default; + + ASTSubquery(ASTPtr child) + { + children.emplace_back(std::move(child)); + } + void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; String getAliasOrColumnName() const override; String tryGetAlias() const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 42400a0f13b..62c480e0f6b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -123,7 +123,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) throw Exception(ErrorCodes::BAD_ARGUMENTS, "EXPLAIN in a subquery cannot have a table function or table override"); /// Replace subquery `(EXPLAIN SELECT ...)` - /// with `(SELECT * FROM viewExplain("", "", SELECT ...))` + /// with `(SELECT * FROM viewExplain('', '', (SELECT ...)))` String kind_str = ASTExplainQuery::toString(explain_query.getKind()); @@ -141,7 +141,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto view_explain = makeASTFunction("viewExplain", std::make_shared(kind_str), std::make_shared(settings_str), - explained_ast); + std::make_shared(explained_ast)); result_node = buildSelectFromTableFunction(view_explain); } else @@ -161,8 +161,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; ++pos; - node = std::make_shared(); - node->children.push_back(result_node); + node = std::make_shared(std::move(result_node)); return true; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1e9383f96ae..6d267a7d215 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -225,8 +225,7 @@ static bool modifyAST(ASTPtr ast, SubqueryFunctionType type) select_with_union_query->list_of_selects->children.push_back(std::move(select_query)); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - auto new_subquery = std::make_shared(); - new_subquery->children.push_back(select_with_union_query); + auto new_subquery = std::make_shared(std::move(select_with_union_query)); ast->children[0]->children.back() = std::move(new_subquery); return true; @@ -1582,8 +1581,7 @@ public: if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) return false; - auto subquery = std::make_shared(); - subquery->children.push_back(std::move(node)); + auto subquery = std::make_shared(std::move(node)); elements = {makeASTFunction("exists", subquery)}; finished = true; diff --git a/src/Parsers/Kusto/ParserKQLQuery.cpp b/src/Parsers/Kusto/ParserKQLQuery.cpp index 47986943662..30e9921e744 100644 --- a/src/Parsers/Kusto/ParserKQLQuery.cpp +++ b/src/Parsers/Kusto/ParserKQLQuery.cpp @@ -576,20 +576,19 @@ bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!ParserKQLTableFunction().parse(pos, select_node, expected)) return false; - ASTPtr node_subquery = std::make_shared(); - node_subquery->children.push_back(select_node); + ASTPtr node_subquery = std::make_shared(std::move(select_node)); ASTPtr node_table_expr = std::make_shared(); node_table_expr->as()->subquery = node_subquery; node_table_expr->children.emplace_back(node_subquery); - ASTPtr node_table_in_select_query_emlement = std::make_shared(); - node_table_in_select_query_emlement->as()->table_expression = node_table_expr; + ASTPtr node_table_in_select_query_element = std::make_shared(); + node_table_in_select_query_element->as()->table_expression = node_table_expr; ASTPtr res = std::make_shared(); - res->children.emplace_back(node_table_in_select_query_emlement); + res->children.emplace_back(node_table_in_select_query_element); node = res; return true; @@ -618,20 +617,19 @@ bool ParserSimpleCHSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ASTSelectQuery::Expression::TABLES, parent_select_node->as()->tables()); } - ASTPtr node_subquery = std::make_shared(); - node_subquery->children.push_back(sub_select_node); + ASTPtr node_subquery = std::make_shared(std::move(sub_select_node)); ASTPtr node_table_expr = std::make_shared(); node_table_expr->as()->subquery = node_subquery; node_table_expr->children.emplace_back(node_subquery); - ASTPtr node_table_in_select_query_emlement = std::make_shared(); - node_table_in_select_query_emlement->as()->table_expression = node_table_expr; + ASTPtr node_table_in_select_query_element = std::make_shared(); + node_table_in_select_query_element->as()->table_expression = node_table_expr; ASTPtr res = std::make_shared(); - res->children.emplace_back(node_table_in_select_query_emlement); + res->children.emplace_back(node_table_in_select_query_element); node = res; return true; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 588e5303117..181fd0ac61c 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -251,8 +251,7 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ view_name = table_expression->database_and_table_name; table_expression->database_and_table_name = {}; - table_expression->subquery = std::make_shared(); - table_expression->subquery->children.push_back(view_query); + table_expression->subquery = std::make_shared(view_query); table_expression->subquery->setAlias(alias); for (auto & child : table_expression->children) diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index f993a9820cb..400fc81e6d4 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -21,6 +22,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int UNEXPECTED_AST_STRUCTURE; } namespace @@ -103,11 +105,25 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt if (function->arguments->children.size() > 2) { - const auto & query_arg = function->arguments->children[2]; + const auto & subquery_arg = function->arguments->children[2]; + const auto * subquery = subquery_arg->as(); + + if (!subquery) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table function '{}' requires a subquery argument, got '{}'", + getName(), queryToString(subquery_arg)); + + if (subquery->children.empty()) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, + "A subquery AST element must have a child"); + + const auto & query_arg = subquery->children[0]; + if (!query_arg->as()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table function '{}' requires a EXPLAIN SELECT query argument, got EXPLAIN '{}'", + "Table function '{}' requires a EXPLAIN's SELECT query argument, got '{}'", getName(), queryToString(query_arg)); + explain_query->setExplainedQuery(query_arg); } else if (kind != ASTExplainQuery::ExplainKind::CurrentTransaction) From 543b07639c97b08b291d76807452c17009cbde6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 05:39:55 +0100 Subject: [PATCH 085/105] Add a test --- .../02990_format_select_from_explain.reference | 9 +++++++++ .../0_stateless/02990_format_select_from_explain.sh | 7 +++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02990_format_select_from_explain.reference create mode 100755 tests/queries/0_stateless/02990_format_select_from_explain.sh diff --git a/tests/queries/0_stateless/02990_format_select_from_explain.reference b/tests/queries/0_stateless/02990_format_select_from_explain.reference new file mode 100644 index 00000000000..7c8dcef3824 --- /dev/null +++ b/tests/queries/0_stateless/02990_format_select_from_explain.reference @@ -0,0 +1,9 @@ +SELECT explain +FROM +( + SELECT * + FROM viewExplain('EXPLAIN AST', '', ( + SELECT * + FROM system.numbers + )) +) diff --git a/tests/queries/0_stateless/02990_format_select_from_explain.sh b/tests/queries/0_stateless/02990_format_select_from_explain.sh new file mode 100755 index 00000000000..4955b733788 --- /dev/null +++ b/tests/queries/0_stateless/02990_format_select_from_explain.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_FORMAT} --query "SELECT explain FROM (EXPLAIN AST SELECT * FROM system.numbers)" From c8415ddc295b2c7f0ebe80aa16a090a6e4697f87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 09:29:34 +0100 Subject: [PATCH 086/105] Speed up the CI --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 9e854dce65a..8858e12c50e 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -190,7 +190,7 @@ function setup_logs_replication echo -e "Creating remote destination table ${table}_${hash} with statement:\n${statement}" >&2 echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ - --distributed_ddl_task_timeout=30 \ + --distributed_ddl_task_timeout=30 --distributed_ddl_output_mode=throw_only_active \ "${CONNECTION_ARGS[@]}" || continue echo "Creating table system.${table}_sender" >&2 From 096a679d6b6f496eebc7e49b2fe0b9ca5b28b8ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 10:47:18 +0100 Subject: [PATCH 087/105] Insignificant changes --- src/Common/Dwarf.cpp | 5 +++-- src/Functions/addressToLine.cpp | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index a405f73e35e..99da3b75429 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -2067,8 +2067,8 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro if (opcode != 0) { // standard opcode // Only interpret opcodes that are recognized by the version we're parsing; - // the others are vendor extensions and we should ignore them. - switch (opcode) // NOLINT(bugprone-switch-missing-default-case) + // the others are vendor extensions, and we should ignore them. + switch (opcode) { case DW_LNS_copy: basicBlock_ = false; @@ -2121,6 +2121,7 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro } isa_ = readULEB(program); return CONTINUE; + default: } // Unrecognized standard opcode, slurp the appropriate number of LEB diff --git a/src/Functions/addressToLine.cpp b/src/Functions/addressToLine.cpp index 771c85cabf6..bb5edf2a07a 100644 --- a/src/Functions/addressToLine.cpp +++ b/src/Functions/addressToLine.cpp @@ -17,7 +17,7 @@ namespace DB namespace { -class FunctionAddressToLine: public FunctionAddressToLineBase +class FunctionAddressToLine : public FunctionAddressToLineBase { public: static constexpr auto name = "addressToLine"; From c2555dfb27c999c04a55ea5896b03b0094b8bf2f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Feb 2024 10:58:10 +0100 Subject: [PATCH 088/105] Better check for inconsistent formatting --- src/Interpreters/executeQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 10567fbafd4..1ab66f49ac7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -721,7 +721,7 @@ static std::tuple executeQueryImpl( /// Verify that AST formatting is consistent: /// If you format AST, parse it back, and format it again, you get the same string. - String formatted1 = ast->formatForErrorMessage(); + String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true); ASTPtr ast2 = parseQuery(parser, formatted1.data(), @@ -730,7 +730,7 @@ static std::tuple executeQueryImpl( chassert(ast2); - String formatted2 = ast2->formatForErrorMessage(); + String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true); if (formatted1 != formatted2) throw Exception(ErrorCodes::LOGICAL_ERROR, From 8f29320a73c394357b20495433a1ac919f8be9c6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 18 Feb 2024 13:15:24 +0100 Subject: [PATCH 089/105] Flush StorageBuffer into multiple threads if num_layers > 1 --- src/Common/CurrentMetrics.cpp | 3 +++ src/Storages/StorageBuffer.cpp | 12 +++++++++++- src/Storages/StorageBuffer.h | 2 ++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index c6fbafa8dc3..6931001202d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -262,6 +262,9 @@ M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \ M(RefreshableViews, "Number materialized views with periodic refreshing (REFRESH)") \ M(RefreshingViews, "Number of materialized views currently executing a refresh") \ + M(StorageBufferFlushThreads, "Number of threads for background flushes in StorageBuffer") \ + M(StorageBufferFlushThreadsActive, "Number of threads for background flushes in StorageBuffer running a task") \ + M(StorageBufferFlushThreadsScheduled, "Number of queued or active threads for background flushes in StorageBuffer") #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index d5c135bb81d..dbf6c7c7657 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -56,6 +56,9 @@ namespace CurrentMetrics { extern const Metric StorageBufferRows; extern const Metric StorageBufferBytes; + extern const Metric StorageBufferFlushThreads; + extern const Metric StorageBufferFlushThreadsActive; + extern const Metric StorageBufferFlushThreadsScheduled; } @@ -131,6 +134,7 @@ StorageBuffer::StorageBuffer( : IStorage(table_id_) , WithContext(context_->getBufferContext()) , num_shards(num_shards_) + , flush_pool(CurrentMetrics::StorageBufferFlushThreads, CurrentMetrics::StorageBufferFlushThreadsActive, CurrentMetrics::StorageBufferFlushThreadsScheduled, num_shards, 0, num_shards_) , buffers(num_shards_) , min_thresholds(min_thresholds_) , max_thresholds(max_thresholds_) @@ -802,7 +806,13 @@ bool StorageBuffer::checkThresholdsImpl(bool direct, size_t rows, size_t bytes, void StorageBuffer::flushAllBuffers(bool check_thresholds) { for (auto & buf : buffers) - flushBuffer(buf, check_thresholds, false); + { + flush_pool.scheduleOrThrowOnError([&] () + { + flushBuffer(buf, check_thresholds, false); + }); + } + flush_pool.wait(); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 47f6239b173..50f12be5aef 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -149,6 +150,7 @@ private: /// There are `num_shards` of independent buffers. const size_t num_shards; + ThreadPool flush_pool; std::vector buffers; const Thresholds min_thresholds; From 29e3e7cb965b5ada347028282e87005c570d3400 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 18 Feb 2024 13:19:35 +0100 Subject: [PATCH 090/105] Better if only 1 layer --- src/Storages/StorageBuffer.cpp | 22 ++++++++++++++++++---- src/Storages/StorageBuffer.h | 2 +- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index dbf6c7c7657..c2b63101d11 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -134,7 +134,6 @@ StorageBuffer::StorageBuffer( : IStorage(table_id_) , WithContext(context_->getBufferContext()) , num_shards(num_shards_) - , flush_pool(CurrentMetrics::StorageBufferFlushThreads, CurrentMetrics::StorageBufferFlushThreadsActive, CurrentMetrics::StorageBufferFlushThreadsScheduled, num_shards, 0, num_shards_) , buffers(num_shards_) , min_thresholds(min_thresholds_) , max_thresholds(max_thresholds_) @@ -157,6 +156,12 @@ StorageBuffer::StorageBuffer( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + if (num_shards > 1) + { + flush_pool = std::make_unique( + CurrentMetrics::StorageBufferFlushThreads, CurrentMetrics::StorageBufferFlushThreadsActive, CurrentMetrics::StorageBufferFlushThreadsScheduled, + num_shards, 0, num_shards); + } flush_handle = bg_pool.createTask(log->name() + "/Bg", [this]{ backgroundFlush(); }); } @@ -807,12 +812,21 @@ void StorageBuffer::flushAllBuffers(bool check_thresholds) { for (auto & buf : buffers) { - flush_pool.scheduleOrThrowOnError([&] () + if (flush_pool) + { + flush_pool->scheduleOrThrowOnError([&] () + { + flushBuffer(buf, check_thresholds, false); + }); + } + else { flushBuffer(buf, check_thresholds, false); - }); + } } - flush_pool.wait(); + + if (flush_pool) + flush_pool->wait(); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 50f12be5aef..6c15c7e0238 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -150,7 +150,7 @@ private: /// There are `num_shards` of independent buffers. const size_t num_shards; - ThreadPool flush_pool; + std::unique_ptr flush_pool; std::vector buffers; const Thresholds min_thresholds; From 6fd563df523006946a5ebc44f350114d541aae56 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 02:58:51 +0100 Subject: [PATCH 091/105] Miscellaneous --- src/AggregateFunctions/AggregateFunctionCount.h | 2 +- src/AggregateFunctions/AggregateFunctionFactory.cpp | 4 ++-- .../Combinators/AggregateFunctionIf.cpp | 2 +- .../Combinators/AggregateFunctionNull.h | 2 +- src/Client/MultiplexedConnections.cpp | 2 +- src/Columns/ColumnArray.cpp | 4 ++-- src/Columns/ColumnNullable.cpp | 6 +----- src/Columns/getLeastSuperColumn.cpp | 2 +- src/Common/SensitiveDataMasker.cpp | 2 +- src/Common/SipHash.h | 2 +- src/Common/checkStackSize.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/DataTypes/DataTypeAggregateFunction.cpp | 2 +- src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp | 2 +- src/Functions/EmptyImpl.h | 2 +- src/Functions/FunctionsComparison.h | 6 +++--- src/Functions/FunctionsLogical.cpp | 2 +- src/Functions/trap.cpp | 2 +- src/Interpreters/Aggregator.cpp | 6 +++--- src/Interpreters/ArrayJoinedColumnsVisitor.h | 2 +- src/Interpreters/ClientInfo.cpp | 4 ++-- src/Interpreters/CrossToInnerJoinVisitor.cpp | 4 ++-- src/Interpreters/DatabaseAndTableWithAlias.cpp | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/InJoinSubqueriesPreprocessor.cpp | 4 ++-- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 2 +- src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 4 ++-- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/ProcessList.cpp | 4 ++-- src/Interpreters/Set.cpp | 4 ++-- src/Interpreters/SetVariants.cpp | 2 +- src/Interpreters/TablesStatus.cpp | 2 +- src/Interpreters/TranslateQualifiedNamesVisitor.cpp | 2 +- src/Interpreters/evaluateConstantExpression.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp | 2 +- .../Formats/RowInputFormatWithDiagnosticInfo.cpp | 2 +- src/Processors/Sources/WaitForAsyncInsertSource.h | 2 +- src/Processors/Transforms/CreatingSetsTransform.cpp | 2 +- src/Processors/Transforms/getSourceFromASTInsertQuery.cpp | 6 +++--- src/QueryPipeline/ExecutionSpeedLimits.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp | 8 ++++---- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageLog.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- src/Storages/StorageView.cpp | 8 ++++---- src/Storages/System/StorageSystemStackTrace.cpp | 2 +- src/Storages/transformQueryForExternalDatabase.cpp | 2 +- utils/check-style/check-style | 5 +++++ 55 files changed, 91 insertions(+), 90 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 36a8617ba91..f5d6030a777 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -219,7 +219,7 @@ public: : IAggregateFunctionDataHelper({argument}, params, createResultType()) { if (!argument->isNullable()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not Nullable data type passed to AggregateFunctionCountNotNullUnary"); } String getName() const override { return "count"; } diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index b6ba562045d..18edb7c8ce0 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -100,7 +100,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( { AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null"); if (!combinator) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot find aggregate function combinator " + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find aggregate function combinator " "to apply a function to Nullable arguments."); DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality); @@ -123,7 +123,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( auto with_original_arguments = getImpl(name, action, types_without_low_cardinality, parameters, out_properties, false); if (!with_original_arguments) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: AggregateFunctionFactory returned nullptr"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory returned nullptr"); return with_original_arguments; } diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp index 20a4bde6bb4..9b5ee79a533 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp @@ -249,7 +249,7 @@ public: : Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionIfNullVariadic"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionIfNullVariadic"); if (number_of_arguments > MAX_ARGS) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionNull.h b/src/AggregateFunctions/Combinators/AggregateFunctionNull.h index 6b6580bf4c4..c8574e82be5 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionNull.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionNull.h @@ -429,7 +429,7 @@ public: , number_of_arguments(arguments.size()) { if (number_of_arguments == 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionNullVariadic"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionNullVariadic"); if (number_of_arguments > MAX_ARGS) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index c7d7d0143c8..8433c8afe9f 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -320,7 +320,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac ReplicaState & state = getReplicaForReading(); current_connection = state.connection; if (current_connection == nullptr) - throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica"); + throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "No available replica"); Packet packet; try diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6f60ec0e642..b3376b35b2e 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -810,7 +810,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint size_t tuple_size = tuple.tupleSize(); if (tuple_size == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple"); Columns temporary_arrays(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -1263,7 +1263,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const size_t tuple_size = tuple.tupleSize(); if (tuple_size == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple"); Columns temporary_arrays(tuple_size); for (size_t i = 0; i < tuple_size; ++i) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 93638371b84..ddf5fc696fb 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -28,7 +26,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; - extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; extern const int NOT_IMPLEMENTED; } @@ -829,8 +826,7 @@ void ColumnNullable::applyNullMap(const ColumnNullable & other) void ColumnNullable::checkConsistency() const { if (null_map->size() != getNestedColumn().size()) - throw Exception(ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT, - "Logical error: Sizes of nested column and null map of Nullable column are not equal"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of nested column and null map of Nullable column are not equal"); } ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const diff --git a/src/Columns/getLeastSuperColumn.cpp b/src/Columns/getLeastSuperColumn.cpp index 6ec5ca7a9c1..4f4a5f2b9b8 100644 --- a/src/Columns/getLeastSuperColumn.cpp +++ b/src/Columns/getLeastSuperColumn.cpp @@ -21,7 +21,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b) ColumnWithTypeAndName getLeastSuperColumn(const std::vector & columns) { if (columns.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no src columns for supercolumn"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No src columns for supercolumn"); ColumnWithTypeAndName result = *columns[0]; diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 70346919f65..28eae6f451d 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -91,7 +91,7 @@ void SensitiveDataMasker::setInstance(std::unique_ptr&& sen { if (!sensitive_data_masker_) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: the 'sensitive_data_masker' is not set"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "The 'sensitive_data_masker' is not set"); if (sensitive_data_masker_->rulesCount() > 0) { diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 5f27fdaa4b6..729fb76a573 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -209,7 +209,7 @@ public: { if (!is_reference_128) throw DB::Exception( - DB::ErrorCodes::LOGICAL_ERROR, "Logical error: can't call get128Reference when is_reference_128 is not set"); + DB::ErrorCodes::LOGICAL_ERROR, "Can't call get128Reference when is_reference_128 is not set"); finalize(); const auto lo = v0 ^ v1 ^ v2 ^ v3; v1 ^= 0xdd; diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 8c2a0aaed7f..1f4faf58fa5 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -136,7 +136,7 @@ __attribute__((__weak__)) void checkStackSize() /// We assume that stack grows towards lower addresses. And that it starts to grow from the end of a chunk of memory of max_stack_size. if (int_frame_address > int_stack_address + max_stack_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: frame address is greater than stack begin address"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Frame address is greater than stack begin address"); size_t stack_size = int_stack_address + max_stack_size - int_frame_address; size_t max_stack_size_allowed = static_cast(max_stack_size * STACK_SIZE_FREE_RATIO); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index c62ed90e378..eaa0c3c9e68 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1583,7 +1583,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc { auto path_prefix = request.path; if (path_prefix.empty()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: path cannot be empty"); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); const auto & children = node_it->value.getChildren(); response.names.reserve(children.size()); diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 7dc036cafa4..14a3c6a4248 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -239,7 +239,7 @@ static DataTypePtr create(const ASTPtr & arguments) argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); if (function_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed"); AggregateFunctionProperties properties; AggregateFunctionPtr function = AggregateFunctionFactory::instance().get(function_name, action, argument_types, params_row, properties); diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index aa3b154e49b..ee9870eb0ef 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -141,7 +141,7 @@ static std::pair create(const ASTPtr & argum argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); if (function_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed"); AggregateFunctionProperties properties; /// NullsAction is not part of the type definition, instead it will have transformed the function into a different one diff --git a/src/Functions/EmptyImpl.h b/src/Functions/EmptyImpl.h index 52484524e6a..d3b2dda024b 100644 --- a/src/Functions/EmptyImpl.h +++ b/src/Functions/EmptyImpl.h @@ -35,7 +35,7 @@ struct EmptyImpl /// Only make sense if is_fixed_to_constant. static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: 'vectorFixedToConstant method' is called"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "'vectorFixedToConstant method' is called"); } static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index d04f76d051a..3be675f39b3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -811,7 +811,7 @@ private: c0_const_size = c0_const_fixed_string->getN(); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column"); } if (c1_const) @@ -830,7 +830,7 @@ private: c1_const_size = c1_const_fixed_string->getN(); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column"); } using StringImpl = StringComparisonImpl>; @@ -1114,7 +1114,7 @@ private: /// This is a paranoid check to protect from a broken query analysis. if (c0->isNullable() != c1->isNullable()) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Logical error: columns are assumed to be of identical types, but they are different in Nullable"); + "Columns are assumed to be of identical types, but they are different in Nullable"); if (c0_const && c1_const) { diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index d01fdc99076..2e0f4cd3038 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -530,7 +530,7 @@ DataTypePtr FunctionAnyArityLogical::getReturnTypeImpl(const DataTyp { has_nullable_arguments = arg_type->isNullable(); if (has_nullable_arguments && !Impl::specialImplementationForNulls()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Unexpected type of argument for function \"{}\": " + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of argument for function \"{}\": " " argument {} is of type {}", getName(), i + 1, arg_type->getName()); } diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 99430f039a4..6ce696fedb5 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -177,7 +177,7 @@ public: } else if (mode == "logical error") { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: trap"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trap"); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown trap mode"); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 331cd991ea1..50fab486568 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -624,7 +624,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_) { size_t alignment_of_next_state = params.aggregates[i + 1].function->alignOfData(); if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: alignOfData is not 2^N"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "`alignOfData` is not 2^N"); /// Extend total_size to next alignment requirement /// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state. @@ -857,7 +857,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() return AggregatedDataVariants::Type::low_cardinality_keys128; if (size_of_field == 32) return AggregatedDataVariants::Type::low_cardinality_keys256; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: low cardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "LowCardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); } if (size_of_field == 1) @@ -872,7 +872,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() return AggregatedDataVariants::Type::keys128; if (size_of_field == 32) return AggregatedDataVariants::Type::keys256; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); } if (params.keys_size == 1 && isFixedString(types_removed_nullable[0])) diff --git a/src/Interpreters/ArrayJoinedColumnsVisitor.h b/src/Interpreters/ArrayJoinedColumnsVisitor.h index 3bbd6982213..f16751c4561 100644 --- a/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -62,7 +62,7 @@ private: { auto [array_join_expression_list, _] = node.arrayJoinExpressionList(); if (!array_join_expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no ARRAY JOIN"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No ARRAY JOIN"); std::vector out; out.reserve(array_join_expression_list->children.size()); diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 347ec115aba..e4778edeb9c 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const { if (server_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::write is called for unsupported server revision"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::write is called for unsupported server revision"); writeBinary(static_cast(query_kind), out); if (empty()) @@ -103,7 +103,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision) { if (client_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::read is called for unsupported client revision"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::read is called for unsupported client revision"); UInt8 read_query_kind = 0; readBinary(read_query_kind, in); diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index 42af164f4ad..e3e8b80e437 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -173,7 +173,7 @@ std::vector getTables(const ASTSelectQuery & select) { const auto * table_element = child->as(); if (!table_element) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: TablesInSelectQueryElement expected"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "TablesInSelectQueryElement expected"); JoinedElement & t = joined_tables.emplace_back(*table_element); t.rewriteCommaToCross(); @@ -224,7 +224,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da { if (joined_tables.size() != data.tables_with_columns.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Logical error: inconsistent number of tables: {} != {}", + "Inconsistent number of tables: {} != {}", joined_tables.size(), data.tables_with_columns.size()); for (size_t i = 0; i < joined_tables.size(); ++i) diff --git a/src/Interpreters/DatabaseAndTableWithAlias.cpp b/src/Interpreters/DatabaseAndTableWithAlias.cpp index db020cb9166..329391b45d7 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -71,7 +71,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & alias = table_expression.subquery->tryGetAlias(); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no known elements in ASTTableExpression"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No known elements in ASTTableExpression"); } bool DatabaseAndTableWithAlias::satisfies(const DatabaseAndTableWithAlias & db_table, bool table_may_be_an_alias) const diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 33dc178ca00..73487a0914a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -368,7 +368,7 @@ HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_c return Type::keys128; if (size_of_field == 32) return Type::keys256; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); } /// If the keys fit in N bits, we will use a hash table for N-bit-packed keys diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 3858830a43b..ec4241a2740 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -103,12 +103,12 @@ private: /// Already processed. } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected function name {}", concrete->name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name {}", concrete->name); } else if (table_join) table_join->locality = JoinLocality::Global; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected AST node"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected AST node"); } else if (distributed_product_mode == DistributedProductMode::DENY) { diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 16bc4b1fe2e..cc1d7dd6531 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -56,7 +56,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( size_t num_children = ast->list_of_selects->children.size(); if (!num_children) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTSelectWithUnionQuery"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTSelectWithUnionQuery"); /// Note that we pass 'required_result_column_names' to first SELECT. /// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT, diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index bf2d1eb79cd..6251a9604e1 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -168,7 +168,7 @@ private: has_asterisks = true; if (!qualified_asterisk->qualifier) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must have a qualifier"); auto & identifier = qualified_asterisk->qualifier->as(); @@ -183,7 +183,7 @@ private: transformer->as()) IASTColumnsTransformer::transform(transformer, columns); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must only have children of IASTColumnsTransformer type"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must only have children of IASTColumnsTransformer type"); } } } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 901c82029ee..d5fb0208d45 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -239,7 +239,7 @@ public: /// SortCursorImpl can work with permutation, but MergeJoinCursor can't. if (impl.permutation) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: MergeJoinCursor doesn't support permutation"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinCursor doesn't support permutation"); } size_t position() const { return impl.getRow(); } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 5b3b87114ae..3bd7b2d4206 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -295,7 +295,7 @@ ProcessListEntry::~ProcessListEntry() auto user_process_list_it = parent.user_to_queries.find(user); if (user_process_list_it == parent.user_to_queries.end()) { - LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find user in ProcessList"); + LOG_ERROR(getLogger("ProcessList"), "Cannot find user in ProcessList"); std::terminate(); } @@ -323,7 +323,7 @@ ProcessListEntry::~ProcessListEntry() if (!found) { - LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); + LOG_ERROR(getLogger("ProcessList"), "Cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); std::terminate(); } diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 84260faafd4..8f11754b3be 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -275,7 +275,7 @@ void Set::appendSetElements(SetKeyColumns & holder) void Set::checkIsCreated() const { if (!is_created.load()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Trying to use set before it has been built."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const @@ -283,7 +283,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co size_t num_key_columns = columns.size(); if (0 == num_key_columns) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no columns passed to Set::execute method."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns passed to Set::execute method."); auto res = ColumnUInt8::create(); ColumnUInt8::Container & vec_res = res->getData(); diff --git a/src/Interpreters/SetVariants.cpp b/src/Interpreters/SetVariants.cpp index cd9148a01cf..0fb2e5189d4 100644 --- a/src/Interpreters/SetVariants.cpp +++ b/src/Interpreters/SetVariants.cpp @@ -146,7 +146,7 @@ typename SetVariantsTemplate::Type SetVariantsTemplate::choose return Type::keys128; if (size_of_field == 32) return Type::keys256; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32."); } /// If the keys fit in N bits, we will use a hash table for N-bit-packed keys diff --git a/src/Interpreters/TablesStatus.cpp b/src/Interpreters/TablesStatus.cpp index 005a4515c3a..911a028f813 100644 --- a/src/Interpreters/TablesStatus.cpp +++ b/src/Interpreters/TablesStatus.cpp @@ -35,7 +35,7 @@ void TableStatus::read(ReadBuffer & in) void TablesStatusRequest::write(WriteBuffer & out, UInt64 server_protocol_revision) const { if (server_protocol_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method TablesStatusRequest::write is called for unsupported server revision"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Method TablesStatusRequest::write is called for unsupported server revision"); writeVarUInt(tables.size(), out); for (const auto & table_name : tables) diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 130ce2194fd..3de7e217e53 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -158,7 +158,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data) { if (!node.qualifier) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must have a qualifier"); /// @note it could contain table alias as table name. DatabaseAndTableWithAlias db_and_table(node.qualifier); diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 00d36750cc1..b5c3e00e299 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -106,7 +106,7 @@ std::optional evaluateConstantExpressionImpl(c if (result_column->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Logical error: empty result column after evaluation " + "Empty result column after evaluation " "of constant expression for IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument"); /// Expressions like rand() or now() are not constant diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 62c480e0f6b..486555ae86d 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -934,7 +934,7 @@ bool ParserNumber::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (float_value < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Logical error: token number cannot begin with minus, " + "Token number cannot begin with minus, " "but parsed float number is less than zero."); if (negative) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 0ef19a9c14f..6fa94356cd3 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -179,7 +179,7 @@ void JSONEachRowRowInputFormat::readJSONObject(MutableColumns & columns) else if (column_index == NESTED_FIELD) readNestedData(name_ref.toString(), columns); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal value of column_index"); } else { diff --git a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp index a56c24a740a..fcf338577f8 100644 --- a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp +++ b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -136,7 +136,7 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co auto * curr_position = in->position(); if (curr_position < prev_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: parsing is non-deterministic."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parsing is non-deterministic."); if (isNativeNumber(type) || isDate(type) || isDateTime(type) || isDateTime64(type)) { diff --git a/src/Processors/Sources/WaitForAsyncInsertSource.h b/src/Processors/Sources/WaitForAsyncInsertSource.h index 1029c164941..78af6294202 100644 --- a/src/Processors/Sources/WaitForAsyncInsertSource.h +++ b/src/Processors/Sources/WaitForAsyncInsertSource.h @@ -33,7 +33,7 @@ protected: { auto status = insert_future.wait_for(std::chrono::milliseconds(timeout_ms)); if (status == std::future_status::deferred) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: got future in deferred state"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got future in deferred state"); if (status == std::future_status::timeout) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout ({} ms) exceeded)", timeout_ms); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index cc0b5926e66..eeb8f4a6060 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -163,7 +163,7 @@ void CreatingSetsTransform::startSubquery() done_with_table = !external_table; if ((done_with_set && !set_from_cache) && done_with_table) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: nothing to do with subquery"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Nothing to do with subquery"); if (table_out.initialized()) { diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 6c7c7447070..8a13973b970 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -37,7 +37,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( const auto * ast_insert_query = ast->as(); if (!ast_insert_query) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: query requires data to insert, but it is not INSERT query"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query requires data to insert, but it is not INSERT query"); if (ast_insert_query->infile && context->getApplicationType() == Context::ApplicationType::SERVER) throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Query has infile and was send directly to server"); @@ -47,7 +47,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( if (input_function) throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()"); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: INSERT query requires format to be set"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "INSERT query requires format to be set"); } /// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query. @@ -105,7 +105,7 @@ std::unique_ptr getReadBufferFromASTInsertQuery(const ASTPtr & ast) { const auto * insert_query = ast->as(); if (!insert_query) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: query requires data to insert, but it is not INSERT query"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query requires data to insert, but it is not INSERT query"); if (insert_query->infile) { diff --git a/src/QueryPipeline/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp index f8ae4c76d0f..05fd394db77 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -113,7 +113,7 @@ static bool handleOverflowMode(OverflowMode mode, int code, FormatStringHelper @@ -179,7 +179,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( size_t prefix_size = block_numbers_path.size() + 1 + partitions[i].size() + 1 + path_prefix.size(); const String & path = dynamic_cast(*lock_responses[i]).path_created; if (path.size() <= prefix_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: name of the sequential node is shorter than prefix."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Name of the sequential node is shorter than prefix."); UInt64 number = parse(path.c_str() + prefix_size, path.size() - prefix_size); locks.push_back(LockInfo{path, partitions[i], number}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e14a358745e..5b297de3fda 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -869,7 +869,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (is_optional) return; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Sign column for storage {} is empty", storage); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Sign column for storage {} is empty", storage); } bool miss_column = true; @@ -896,7 +896,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (is_optional) return; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Version column for storage {} is empty", storage); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Version column for storage {} is empty", storage); } bool miss_column = true; @@ -925,12 +925,12 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (is_optional) return; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: is_deleted ({}) column for storage {} is empty", is_deleted_column, storage); + throw Exception(ErrorCodes::LOGICAL_ERROR, "`is_deleted` ({}) column for storage {} is empty", is_deleted_column, storage); } else { if (version_column.empty() && !is_optional) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Version column ({}) for storage {} is empty while is_deleted ({}) is not.", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Version column ({}) for storage {} is empty while is_deleted ({}) is not.", version_column, storage, is_deleted_column); bool miss_is_deleted_column = true; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 58fddde7b54..1bf1d4a3c29 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -85,7 +85,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t max_coun if (scheduled_tasks_count > max_count) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Logical error: invalid argument passed to getMaxSourcePartsSize: scheduled_tasks_count = {} > max_count = {}", + "Invalid argument passed to getMaxSourcePartsSize: scheduled_tasks_count = {} > max_count = {}", scheduled_tasks_count, max_count); } @@ -511,7 +511,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( /// Do not allow to "merge" part with itself for regular merges, unless it is a TTL-merge where it is ok to remove some values with expired ttl if (parts_to_merge.size() == 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: merge selector returned only one part to merge"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge selector returned only one part to merge"); if (parts_to_merge.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c9c16b59f9e..ebf887f5e9e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -384,13 +384,13 @@ Block MergeTreeDataWriter::mergeBlock( /// Check that after first merge merging_algorithm is waiting for data from input 0. if (status.required_source != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: required source after the first merge is not 0. Chunk rows: {}, is_finished: {}, required_source: {}, algorithm: {}", status.chunk.getNumRows(), status.is_finished, status.required_source, merging_algorithm->getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Required source after the first merge is not 0. Chunk rows: {}, is_finished: {}, required_source: {}, algorithm: {}", status.chunk.getNumRows(), status.is_finished, status.required_source, merging_algorithm->getName()); status = merging_algorithm->merge(); /// Check that merge is finished. if (!status.is_finished) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: merge is not finished after the second merge."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge is not finished after the second merge."); /// Merged Block is sorted and we don't need to use permutation anymore permutation = nullptr; @@ -439,7 +439,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( auto max_month = date_lut.toNumYYYYMM(max_date); if (min_month != max_month) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: part spans more than one month."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part spans more than one month."); part_name = new_part_info.getPartNameV0(min_date, max_date); } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 4e339964de3..da89d52a9ff 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -54,9 +54,9 @@ MarkType::MarkType(bool adaptive_, bool compressed_, MergeTreeDataPartType::Valu : adaptive(adaptive_), compressed(compressed_), part_type(part_type_) { if (!adaptive && part_type != MergeTreeDataPartType::Wide) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: non-Wide data part type with non-adaptive granularity"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-Wide data part type with non-adaptive granularity"); if (part_type == MergeTreeDataPartType::Unknown) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown data part type"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown data part type"); } bool MarkType::isMarkFileExtension(std::string_view extension) @@ -71,7 +71,7 @@ std::string MarkType::getFileExtension() const if (!adaptive) { if (part_type != MergeTreeDataPartType::Wide) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: non-Wide data part type with non-adaptive granularity"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-Wide data part type with non-adaptive granularity"); return res; } @@ -84,7 +84,7 @@ std::string MarkType::getFileExtension() const case MergeTreeDataPartType::InMemory: return ""; case MergeTreeDataPartType::Unknown: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown data part type"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown data part type"); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index b9e082c0b22..b122674466f 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -500,7 +500,7 @@ protected: Chunk chunk; if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(), [&](auto kind, auto strictness, auto & map) { chunk = createChunk(map); })) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown JOIN strictness"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness"); return chunk; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c7b0a9d0644..99192fe1e50 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -241,7 +241,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu const auto & data_file_it = storage.data_files_by_names.find(data_file_name); if (data_file_it == storage.data_files_by_names.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no information about file {} in StorageLog", data_file_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name); const auto & data_file = *data_file_it->second; size_t offset = stream_for_prefix ? 0 : offsets[data_file.index]; @@ -448,7 +448,7 @@ ISerialization::OutputStreamGetter LogSink::createStreamGetter(const NameAndType String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); auto it = streams.find(data_file_name); if (it == streams.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: stream was not created when writing data in LogSink"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Stream was not created when writing data in LogSink"); Stream & stream = it->second; if (stream.written) @@ -473,7 +473,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c { const auto & data_file_it = storage.data_files_by_names.find(data_file_name); if (data_file_it == storage.data_files_by_names.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no information about file {} in StorageLog", data_file_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name); const auto & data_file = *data_file_it->second; const auto & columns = metadata_snapshot->getColumns(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6119541ff52..fb4e9b4ad87 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2043,7 +2043,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che if (entry.quorum) { if (entry.type != LogEntry::GET_PART) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: log entry with quorum but type is not GET_PART"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Log entry with quorum but type is not GET_PART"); LOG_DEBUG(log, "No active replica has part {} which needs to be written with quorum. Will try to mark that quorum as failed.", entry.new_part_name); @@ -2106,7 +2106,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); if (part_info.min_block != part_info.max_block) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: log entry with quorum for part covering more than one block number"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Log entry with quorum for part covering more than one block number"); ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name, @@ -6796,7 +6796,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( } else { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected name of log node: {}", entry.znode_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected name of log node: {}", entry.znode_name); } /** Second - find the corresponding entry in the queue of the specified replica. @@ -7172,7 +7172,7 @@ void StorageReplicatedMergeTree::fetchPartition( } if (best_replica.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot choose best replica."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot choose best replica."); LOG_INFO(log, "Found {} replicas, {} of them are active. Selected {} to fetch from.", replicas.size(), active_replicas.size(), best_replica); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 181fd0ac61c..5679effbcb2 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -207,12 +207,12 @@ void StorageView::read( static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) { if (!select_query.tables() || select_query.tables()->children.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no table expression in view select AST"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No table expression in view select AST"); auto * select_element = select_query.tables()->children[0]->as(); if (!select_element->table_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression"); return select_element->table_expression->as(); } @@ -243,7 +243,7 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ } if (!table_expression->database_and_table_name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression"); } DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name); @@ -270,7 +270,7 @@ ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr ASTTableExpression * table_expression = getFirstTableExpression(select_query); if (!table_expression->subquery) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression"); ASTPtr subquery = table_expression->subquery; table_expression->subquery = {}; diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 82a5fd4e33f..90eb0ad89ec 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -168,7 +168,7 @@ bool wait(int timeout_ms) continue; /// Drain delayed notifications. } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: read wrong number of bytes from pipe"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read wrong number of bytes from pipe"); } } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 4526a38a1c3..afc458ea612 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -145,7 +145,7 @@ bool isCompatible(ASTPtr & node) return false; if (!function->arguments) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: function->arguments is not set"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "function->arguments is not set"); String name = function->name; diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6c12970c4bb..a71dac91683 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -448,3 +448,8 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | grep -vP $EXCLUDE_DIRS | xargs grep -P 'Sql|Html|Xml|Cpu|Tcp|Udp|Http|Db|Json|Yaml' | grep -v -P 'RabbitMQ|Azure|Aws|aws|Avro|IO/S3' && echo "Abbreviations such as SQL, XML, HTTP, should be in all caps. For example, SQL is right, Sql is wrong. XMLHttpRequest is very wrong." + +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | + grep -vP $EXCLUDE_DIRS | + xargs grep -F -i 'ErrorCodes::LOGICAL_ERROR, "Logical error:' && + echo "If an exception has LOGICAL_ERROR code, there is no need to include the text 'Logical error' in the exception message, because then the phrase 'Logical error' will be printed twice." From fa1ca348a4326a96a58d74e3ad4bac738c2b22c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 03:19:21 +0100 Subject: [PATCH 092/105] Miscellaneous --- src/Core/MySQL/PacketEndpoint.cpp | 2 +- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 4 ++-- src/Functions/FunctionsConversion.h | 2 +- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Server/HTTPHandlerFactory.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/MySQL/PacketEndpoint.cpp b/src/Core/MySQL/PacketEndpoint.cpp index 97b5d3b4d11..085d7595167 100644 --- a/src/Core/MySQL/PacketEndpoint.cpp +++ b/src/Core/MySQL/PacketEndpoint.cpp @@ -40,7 +40,7 @@ bool PacketEndpoint::tryReceivePacket(IMySQLReadPacket & packet, UInt64 millisec ReadBufferFromPocoSocket * socket_in = typeid_cast(in); if (!socket_in) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: Attempt to pull the duration in a non socket stream"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to pull the duration in a non socket stream"); if (!socket_in->poll(millisecond * 1000)) return false; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 2656835f912..20db8036942 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -779,7 +779,7 @@ static void writeFieldsToColumn( casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: it is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); } } } @@ -844,7 +844,7 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) { if (rows_data.size() % 2 != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); size_t prev_bytes = buffer.bytes(); std::vector writeable_rows_mask(rows_data.size()); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4089a5b542b..62148fa8022 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2560,7 +2560,7 @@ public: if constexpr (std::is_same_v) res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); else if constexpr (std::is_same_v) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); else if constexpr (to_decimal) { UInt64 scale = extractToDecimalScale(arguments[1]); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 0fdc9347ee9..107b435ded4 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -337,7 +337,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) WhichDataType which(type); if (which.isNullable()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: MySQL primary key must be not null, it is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MySQL's primary key must be not null, it is a bug."); if (which.isDate() || which.isDate32() || which.isDateTime() || which.isDateTime64()) { diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 06ca1182be5..9e4a440ddb2 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -120,7 +120,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); } - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: Unknown HTTP handler factory name."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index da49814b83a..f506230b5ea 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -59,7 +59,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & const auto * non_const_column = typeid_cast(hash_column); if (!const_column && !non_const_column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: hash column must be Const Column or UInt64 Column."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); if (const_column) { diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index db85c804d8d..8029d6d405b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -53,7 +53,7 @@ public: if (const auto & bf_granule = typeid_cast(granule.get())) return mayBeTrueOnGranule(bf_granule); - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: require bloom filter index granule."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); } private: From 4590d3f81a1f44f2a8de137a60ea496230cd5d86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 05:10:39 +0100 Subject: [PATCH 093/105] Miscellaneous --- src/Interpreters/getHeaderForProcessingStage.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index d16e01ef2d2..67a909ba6b4 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -167,8 +167,7 @@ Block getHeaderForProcessingStage( return result; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical Error: unknown processed stage."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown processed stage."); } } - From e80bc24163efa5d838c3c9398644c19bba868e42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 05:18:17 +0100 Subject: [PATCH 094/105] Writing object name is useless for our stack traces --- src/Common/StackTrace.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 7e683ae91de..436b85ff30b 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -448,9 +448,6 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s DB::writePointerHex(frame.physical_addr, out); } - if (frame.object.has_value()) - out << " in " << *frame.object; - callback(out.str()); }; #else From e2de74cf66d218032615d68b2b073529ec359c60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 05:34:40 +0100 Subject: [PATCH 095/105] Fix debug --- src/Parsers/IParser.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index e8f95954e70..198ec0346ff 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -82,7 +82,7 @@ public: * The frequency is arbitrary, but not too large, not too small, * and a power of two to simplify the division. */ -#if defined(USE_MUSL) || defined(SANITIZER) +#if defined(USE_MUSL) || defined(SANITIZER) || !defined(NDEBUG) static constexpr uint32_t check_frequency = 128; #else static constexpr uint32_t check_frequency = 8192; From ad26b6fc8d3e773594ced086d3915422071f597c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 07:08:24 +0100 Subject: [PATCH 096/105] Remove useless headers --- src/Client/ConnectionEstablisher.h | 1 - src/Client/PacketReceiver.h | 1 - src/Common/checkStackSize.cpp | 6 ------ src/QueryPipeline/RemoteQueryExecutorReadContext.h | 1 - 4 files changed, 9 deletions(-) diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 1fa08d435e9..a3a01e63246 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index deedf5cccdc..6b3da659290 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 1f4faf58fa5..954def83e94 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -1,12 +1,6 @@ #include #include -#include -#include -#include /// THREAD_SANITIZER -#include #include -#include -#include #if defined(OS_FREEBSD) # include diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index adfc0c5eacf..50df7e2db35 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include From 7fa9c733bd290ab081c151bf64c37f964cb92e8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 07:27:02 +0100 Subject: [PATCH 097/105] Fix Fibers --- src/Common/Fiber.h | 15 +++++++-------- src/Common/checkStackSize.cpp | 5 +++++ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index f48ace149f4..8b88bd323ef 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -17,7 +17,7 @@ private: template friend class FiberLocal; public: - template< typename StackAlloc, typename Fn> + template Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward(salloc), RoutineImpl(std::forward(fn))) { } @@ -46,6 +46,12 @@ public: current_fiber = parent_fiber; } + static FiberPtr & getCurrentFiber() + { + thread_local static FiberPtr current_fiber; + return current_fiber; + } + private: template struct RoutineImpl @@ -74,12 +80,6 @@ private: Fn fn; }; - static FiberPtr & getCurrentFiber() - { - thread_local static FiberPtr current_fiber; - return current_fiber; - } - /// Special wrapper to store data in uniquer_ptr. struct DataWrapper { @@ -146,4 +146,3 @@ private: T main_instance; }; - diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 954def83e94..fd0e8d89ca6 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #if defined(OS_FREEBSD) @@ -108,6 +109,10 @@ __attribute__((__weak__)) void checkStackSize() { using namespace DB; + /// Not implemented for coroutines. + if (Fiber::getCurrentFiber()) + return; + if (!stack_address) max_stack_size = getStackSize(&stack_address); From 6329b6ae459519bc0471d3ce0179ed2bd09dacca Mon Sep 17 00:00:00 2001 From: Hongbin Ma Date: Mon, 19 Feb 2024 14:49:02 +0800 Subject: [PATCH 098/105] fix flaky test case --- tests/queries/0_stateless/00873_t64_codec_date.reference | 2 +- tests/queries/0_stateless/00873_t64_codec_date.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00873_t64_codec_date.reference b/tests/queries/0_stateless/00873_t64_codec_date.reference index 1568c3122e6..9353696610c 100644 --- a/tests/queries/0_stateless/00873_t64_codec_date.reference +++ b/tests/queries/0_stateless/00873_t64_codec_date.reference @@ -1,4 +1,4 @@ 1970-01-01 1970-01-01 1950-01-01 1950-01-01 1970-01-01 1970-01-01 1970-01-01 1970-01-01 -2149-06-06 2149-06-06 2149-06-08 2149-06-08 2149-06-06 2149-06-06 2149-06-06 2149-06-06 +2149-06-06 2149-06-06 2149-06-08 2149-06-08 diff --git a/tests/queries/0_stateless/00873_t64_codec_date.sql b/tests/queries/0_stateless/00873_t64_codec_date.sql index e9230c75665..c6e21baba12 100644 --- a/tests/queries/0_stateless/00873_t64_codec_date.sql +++ b/tests/queries/0_stateless/00873_t64_codec_date.sql @@ -13,7 +13,7 @@ INSERT INTO t64 values ('2149-06-06', '2149-06-06', '2149-06-06', '2149-06-06'); INSERT INTO t64 values ('2149-06-08', '2149-06-08', '2149-06-08', '2149-06-08'); INSERT INTO t64 values ('1950-01-01', '1950-01-01', '1950-01-01', '1950-01-01'); -SELECT * FROM t64 ORDER BY date16; +SELECT * FROM t64 ORDER BY date_32; SELECT * FROM t64 WHERE date16 != t_date16; SELECT * FROM t64 WHERE date_32 != t_date32; From 1f882063e759ec43d42fb47c83fc02b7510bc455 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 08:44:53 +0100 Subject: [PATCH 099/105] Fix build --- src/Common/checkStackSize.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index fd0e8d89ca6..c88554ca8fe 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -1,7 +1,12 @@ +#include +#include /// THREAD_SANITIZER #include #include #include +#include #include +#include +#include #if defined(OS_FREEBSD) # include From 327ba7d174f287bb6a25fefd8372632b31c0f62e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Feb 2024 09:30:19 +0100 Subject: [PATCH 100/105] Fix bad log message --- src/Common/LoggingFormatStringHelpers.cpp | 5 ++--- src/Processors/Formats/Impl/Parquet/Write.cpp | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Common/LoggingFormatStringHelpers.cpp b/src/Common/LoggingFormatStringHelpers.cpp index b3353a59010..7cbef779f28 100644 --- a/src/Common/LoggingFormatStringHelpers.cpp +++ b/src/Common/LoggingFormatStringHelpers.cpp @@ -130,13 +130,12 @@ LogSeriesLimiter::LogSeriesLimiter(LoggerPtr logger_, size_t allowed_count_, tim if (last_time + interval_s_ <= now) { debug_message = fmt::format( - " (LogSeriesLimiter: on interval from {} to {} accepted series {} / {} for the logger {} : {})", + " (LogSeriesLimiter: on interval from {} to {} accepted series {} / {} for the logger {})", DateLUT::instance().timeToString(last_time), DateLUT::instance().timeToString(now), accepted_count, total_count, - logger->name(), - double(name_hash)); + logger->name()); register_as_first(); return; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 02ca2734ff8..4d71e0102d8 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -409,7 +409,7 @@ PODArray & compress(PODArray & source, PODArray & scratch, Com #pragma clang diagnostic pop if (max_dest_size > std::numeric_limits::max()) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", formatReadableSizeWithBinarySuffix(source.size())); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", ReadableSize(source.size())); scratch.resize(max_dest_size); From 6565423b1a3ca7a6127b848fc112e8c2eadb66ae Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Feb 2024 10:32:36 +0100 Subject: [PATCH 101/105] Review fix --- src/Storages/StorageBuffer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c2b63101d11..5d717f84a1d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -814,10 +815,10 @@ void StorageBuffer::flushAllBuffers(bool check_thresholds) { if (flush_pool) { - flush_pool->scheduleOrThrowOnError([&] () + scheduleFromThreadPool([&] () { flushBuffer(buf, check_thresholds, false); - }); + }, *flush_pool, "BufferFlush"); } else { From 0496d0f45fa6d07038a32de93f6f65ed02f0c971 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 18 Feb 2024 10:55:22 +0100 Subject: [PATCH 102/105] Do not retry queries if container is down in integration tests Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1d96563251b..95722dd0db9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3484,6 +3484,10 @@ class ClickHouseInstance: if check_callback(result): return result time.sleep(sleep_time) + except QueryRuntimeException as ex: + # Container is down, this is likely due to server crash. + if "No route to host" in str(ex): + raise except Exception as ex: # logging.debug("Retry {} got exception {}".format(i + 1, ex)) time.sleep(sleep_time) From cfa198c4773cf05619766020bbe6aee859bb2739 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 19 Feb 2024 13:34:00 +0100 Subject: [PATCH 103/105] remove comment Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Functions/FunctionBinaryArithmetic.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 9b4249b0aef..d253095ca01 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -2101,7 +2101,6 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A type_res = std::make_shared(left.getPrecision(), left.getScale()); else type_res = std::make_shared(right.getPrecision(), right.getScale()); - // Create result decimal type somehow, maybe similar to how we do it in getReturnTypeImplStatic auto res = executeNumericWithDecimal( left, right, From a5b62df0d6a2202d4a8ccdd766470db1c77e3cc3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Feb 2024 13:46:51 +0100 Subject: [PATCH 104/105] Update 02982_dont_infer_exponent_floats.sql --- tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql index 17f62557fc2..2a281e898f1 100644 --- a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql +++ b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql @@ -1,3 +1,2 @@ DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 0; DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 1; - From 9361946d151a082ca190b9d7489804b9c30ef3b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 19 Feb 2024 17:48:53 +0100 Subject: [PATCH 105/105] Fix build in master --- src/Storages/StorageBuffer.cpp | 58 +++++++++++++++++----------------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 5d717f84a1d..2925038ec8e 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1,41 +1,41 @@ -#include -#include #include +#include +#include #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include +#include +#include +#include #include #include #include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace ProfileEvents