From 1b2357198d4c1b92fbe3f5a6c0c11be6f1d80a8f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Mar 2024 17:03:43 +0000 Subject: [PATCH 01/19] Fix: parallel replicas + prewhere --- src/Interpreters/InterpreterSelectQuery.cpp | 5 ++-- ...03006_parallel_replicas_prewhere.reference | 0 .../03006_parallel_replicas_prewhere.sql | 29 +++++++++++++++++++ 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference create mode 100644 tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index eaa256a16b0..80e2d5afef7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2112,7 +2112,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. analysis.prewhere_info - = std::make_shared(std::move(analysis.filter_info->actions), std::move(analysis.filter_info->column_name)); + = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); analysis.prewhere_info->prewhere_actions->projectInput(false); analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; analysis.prewhere_info->need_filter = true; @@ -2121,8 +2121,9 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis } else { + chassert(analysis.filter_info->actions); /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.prewhere_info->row_level_filter->projectInput(false); analysis.filter_info = nullptr; diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql new file mode 100644 index 00000000000..819526b5596 --- /dev/null +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -0,0 +1,29 @@ +DROP POLICY IF EXISTS url_na_log_policy0 ON url_na_log; +DROP TABLE IF EXISTS url_na_log; + +CREATE TABLE url_na_log +( + `SiteId` UInt32, + `DateVisit` Date +) +ENGINE = MergeTree +PRIMARY KEY SiteId +ORDER BY (SiteId, DateVisit) +SETTINGS index_granularity = 1000, min_bytes_for_wide_part = 0; + +CREATE ROW POLICY url_na_log_policy0 ON url_na_log FOR SELECT USING (DateVisit < '2022-08-11') OR (DateVisit > '2022-08-19') TO default; + +INSERT INTO url_na_log +SETTINGS max_insert_block_size = 200000 +SELECT + 209, + CAST('2022-08-09', 'Date') + toIntervalDay(intDiv(number, 10000)) +FROM numbers(130000) +SETTINGS max_insert_block_size = 200000; + +EXPLAIN ESTIMATE +SELECT count() +FROM url_na_log +PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') +SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, allow_experimental_analyzer=0, parallel_replicas_min_number_of_rows_per_replica=10000; + From a2bdb054ca5a9c205f5d0442313d3c9f4d22cda7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Mar 2024 17:31:37 +0000 Subject: [PATCH 02/19] Cleanup --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +--- .../queries/0_stateless/03006_parallel_replicas_prewhere.sql | 4 +++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 80e2d5afef7..987bf560add 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2111,8 +2111,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. - analysis.prewhere_info - = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); + analysis.prewhere_info = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); analysis.prewhere_info->prewhere_actions->projectInput(false); analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; analysis.prewhere_info->need_filter = true; @@ -2121,7 +2120,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis } else { - chassert(analysis.filter_info->actions); /// Add row level security actions to prewhere. analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index 819526b5596..4b84646c034 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -25,5 +25,7 @@ EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, allow_experimental_analyzer=0, parallel_replicas_min_number_of_rows_per_replica=10000; +SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; +DROP POLICY url_na_log_policy0 ON url_na_log; +DROP TABLE url_na_log; From 46332f334ccda467abfa673a9e75683a47e24110 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Fri, 8 Mar 2024 15:17:07 +0200 Subject: [PATCH 03/19] rm unused alias set/vector pair --- src/Client/QueryFuzzer.cpp | 16 ---------------- src/Client/QueryFuzzer.h | 4 +--- 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 0597a7c1eed..ea2e5e17afd 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1227,12 +1227,6 @@ void QueryFuzzer::collectFuzzInfoMain(ASTPtr ast) { collectFuzzInfoRecurse(ast); - aliases.clear(); - for (const auto & alias : aliases_set) - { - aliases.push_back(alias); - } - column_like.clear(); for (const auto & [name, value] : column_like_map) { @@ -1285,16 +1279,6 @@ void QueryFuzzer::addColumnLike(ASTPtr ast) void QueryFuzzer::collectFuzzInfoRecurse(ASTPtr ast) { - if (auto * impl = dynamic_cast(ast.get())) - { - if (aliases_set.size() > 1000) - { - aliases_set.clear(); - } - - aliases_set.insert(impl->alias); - } - if (typeid_cast(ast.get())) { addColumnLike(ast); diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index f5465626d96..3bc7b0842d3 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -50,9 +50,7 @@ struct QueryFuzzer // we are currently fuzzing. We add some part from each new query we are asked // to fuzz, and keep this state between queries, so the fuzzing output becomes // more interesting over time, as the queries mix. - std::unordered_set aliases_set; - std::vector aliases; - + // The maps are used for collection, and the vectors are used for random access. std::unordered_map column_like_map; std::vector column_like; From c090c6fb0a6a91bb530654bf96bb31acb9d19f91 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 8 Mar 2024 18:52:15 +0000 Subject: [PATCH 04/19] Fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 987bf560add..bcedba7346d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2122,7 +2122,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis { /// Add row level security actions to prewhere. analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; - analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); + analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name; analysis.prewhere_info->row_level_filter->projectInput(false); analysis.filter_info = nullptr; } From 47b308d2346ef2b87d478d8ee29ee439097df000 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Mar 2024 08:42:33 +0100 Subject: [PATCH 05/19] Simplify bridges --- .../library-bridge/LibraryBridgeHandlers.cpp | 3 -- programs/odbc-bridge/ColumnInfoHandler.cpp | 2 - .../odbc-bridge/IdentifierQuoteHandler.cpp | 3 -- programs/odbc-bridge/ODBCBlockInputStream.cpp | 50 ++++++------------- .../odbc-bridge/ODBCBlockOutputStream.cpp | 1 - 5 files changed, 16 insertions(+), 43 deletions(-) diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index ab146f458df..26d887cfc98 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -1,6 +1,5 @@ #include "LibraryBridgeHandlers.h" -#include "CatBoostLibraryHandler.h" #include "CatBoostLibraryHandlerFactory.h" #include "Common/ProfileEvents.h" #include "ExternalDictionaryLibraryHandler.h" @@ -11,10 +10,8 @@ #include #include #include -#include #include #include -#include #include #include #include diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 774883657b7..4cb15de3b2c 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -8,13 +8,11 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include #include "getIdentifierQuote.h" diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index a23efb112de..cf5acdc4534 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -7,13 +7,10 @@ #include #include #include -#include -#include #include #include #include #include -#include #include "getIdentifierQuote.h" #include "validateODBCConnectionString.h" #include "ODBCPooledConnectionFactory.h" diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index c46144c3dc8..2cccc66a033 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -3,12 +3,8 @@ #include #include #include -#include -#include -#include #include #include -#include #include @@ -54,21 +50,7 @@ Chunk ODBCSource::generate() const auto & sample = description.sample_block.getByPosition(idx); if (!result.is_null(idx)) - { - bool is_nullable = description.types[idx].second; - - if (is_nullable) - { - ColumnNullable & column_nullable = assert_cast(*columns[idx]); - const auto & data_type = assert_cast(*sample.type); - insertValue(column_nullable.getNestedColumn(), data_type.getNestedType(), description.types[idx].first, result, idx); - column_nullable.getNullMapData().emplace_back(0); - } - else - { - insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); - } - } + insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); else insertDefaultValue(*columns[idx], *sample.column); } @@ -87,49 +69,49 @@ void ODBCSource::insertValue( switch (type) { case ValueType::vtUInt8: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt16: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUInt64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt8: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt16: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtInt64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFloat32: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFloat64: - assert_cast(column).insertValue(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtFixedString:[[fallthrough]]; case ValueType::vtEnum8: case ValueType::vtEnum16: case ValueType::vtString: - assert_cast(column).insert(row.get(idx)); + column.insert(row.get(idx)); break; case ValueType::vtUUID: { auto value = row.get(idx); - assert_cast(column).insert(parse(value.data(), value.size())); + column.insert(parse(value.data(), value.size())); break; } case ValueType::vtDate: - assert_cast(column).insertValue(UInt16{LocalDate{row.get(idx)}.getDayNum()}); + column.insert(UInt16{LocalDate{row.get(idx)}.getDayNum()}); break; case ValueType::vtDateTime: { @@ -139,7 +121,7 @@ void ODBCSource::insertValue( readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast(column).insertValue(static_cast(time)); + column.insert(static_cast(time)); break; } case ValueType::vtDateTime64: @@ -149,7 +131,7 @@ void ODBCSource::insertValue( DateTime64 time = 0; const auto * datetime_type = assert_cast(data_type.get()); readDateTime64Text(time, datetime_type->getScale(), in, datetime_type->getTimeZone()); - assert_cast(column).insertValue(time); + column.insert(time); break; } case ValueType::vtDecimal32: [[fallthrough]]; diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 87c09d1e757..37b70023169 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -1,6 +1,5 @@ #include "ODBCBlockOutputStream.h" -#include #include #include #include From 4af16043b86a6e72b3f13579a938d271fd0b77dd Mon Sep 17 00:00:00 2001 From: tomershafir Date: Sat, 9 Mar 2024 21:27:28 +0200 Subject: [PATCH 06/19] improve randomness by erase buf elem when full --- src/Client/QueryFuzzer.cpp | 49 +++++++++++++++++++------------------- src/Client/QueryFuzzer.h | 6 ++--- 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index ea2e5e17afd..d23b5159854 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -232,7 +232,7 @@ ASTPtr QueryFuzzer::getRandomColumnLike() return nullptr; } - ASTPtr new_ast = column_like[fuzz_rand() % column_like.size()]->clone(); + ASTPtr new_ast = column_like[fuzz_rand() % column_like.size()].second->clone(); new_ast->setAlias(""); return new_ast; @@ -272,7 +272,7 @@ void QueryFuzzer::replaceWithTableLike(ASTPtr & ast) return; } - ASTPtr new_ast = table_like[fuzz_rand() % table_like.size()]->clone(); + ASTPtr new_ast = table_like[fuzz_rand() % table_like.size()].second->clone(); std::string old_alias = ast->tryGetAlias(); new_ast->setAlias(old_alias); @@ -1214,51 +1214,46 @@ void QueryFuzzer::fuzz(ASTPtr & ast) } } +#define AST_FUZZER_PART_TYPE_CAP 1000 + /* * This functions collects various parts of query that we can then substitute * to a query being fuzzed. - * - * TODO: we just stop remembering new parts after our corpus reaches certain size. - * This is boring, should implement a random replacement of existing parst with - * small probability. Do this after we add this fuzzer to CI and fix all the - * problems it can routinely find even in this boring version. */ void QueryFuzzer::collectFuzzInfoMain(ASTPtr ast) { collectFuzzInfoRecurse(ast); - - column_like.clear(); - for (const auto & [name, value] : column_like_map) - { - column_like.push_back(value); - } - - table_like.clear(); - for (const auto & [name, value] : table_like_map) - { - table_like.push_back(value); - } } void QueryFuzzer::addTableLike(ASTPtr ast) { - if (table_like_map.size() > 1000) + if (table_like_map.size() > AST_FUZZER_PART_TYPE_CAP) { - table_like_map.clear(); + const auto iter = std::next(table_like.begin(), fuzz_rand() % table_like.size()); + const auto ast_del = *iter; + table_like.erase(iter); + table_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); if (name.size() < 200) { - table_like_map.insert({name, ast}); + const auto res = table_like_map.insert({name, ast}); + if (res.second) + { + table_like.push_back({name, ast}); + } } } void QueryFuzzer::addColumnLike(ASTPtr ast) { - if (column_like_map.size() > 1000) + if (column_like_map.size() > AST_FUZZER_PART_TYPE_CAP) { - column_like_map.clear(); + const auto iter = std::next(column_like.begin(), fuzz_rand() % column_like.size()); + const auto ast_del = *iter; + column_like.erase(iter); + column_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); @@ -1273,7 +1268,11 @@ void QueryFuzzer::addColumnLike(ASTPtr ast) } if (name.size() < 200) { - column_like_map.insert({name, ast}); + const auto res = column_like_map.insert({name, ast}); + if (res.second) + { + column_like.push_back({name, ast}); + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Client/QueryFuzzer.h index 3bc7b0842d3..6165e589cae 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Client/QueryFuzzer.h @@ -50,12 +50,12 @@ struct QueryFuzzer // we are currently fuzzing. We add some part from each new query we are asked // to fuzz, and keep this state between queries, so the fuzzing output becomes // more interesting over time, as the queries mix. - // The maps are used for collection, and the vectors are used for random access. + // The hash tables are used for collection, and the vectors are used for random access. std::unordered_map column_like_map; - std::vector column_like; + std::vector> column_like; std::unordered_map table_like_map; - std::vector table_like; + std::vector> table_like; // Some debug fields for detecting problematic ASTs with loops. // These are reset for each fuzzMain call. From 453eb259842f3181d5f9329e0e82e4c1e66f18b0 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Sun, 10 Mar 2024 08:42:54 +0200 Subject: [PATCH 07/19] rm trailing whitespace --- src/Client/QueryFuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index d23b5159854..7f1dce4b29a 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1232,7 +1232,7 @@ void QueryFuzzer::addTableLike(ASTPtr ast) const auto iter = std::next(table_like.begin(), fuzz_rand() % table_like.size()); const auto ast_del = *iter; table_like.erase(iter); - table_like_map.erase(ast_del.first); + table_like_map.erase(ast_del.first); } const auto name = ast->formatForErrorMessage(); From 9ddec346a03108198fea011c1113495050ac628b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:02:34 +0100 Subject: [PATCH 08/19] Miscellaneous --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 8 +- src/Storages/MergeTree/MergeTask.cpp | 4 +- .../test_odbc_interaction/test_exiled.py | 111 ------------------ 3 files changed, 6 insertions(+), 117 deletions(-) delete mode 100644 tests/integration/test_odbc_interaction/test_exiled.py diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 2cccc66a033..4fd9b4cca45 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -98,7 +98,7 @@ void ODBCSource::insertValue( case ValueType::vtFloat64: column.insert(row.get(idx)); break; - case ValueType::vtFixedString:[[fallthrough]]; + case ValueType::vtFixedString: case ValueType::vtEnum8: case ValueType::vtEnum16: case ValueType::vtString: @@ -134,9 +134,9 @@ void ODBCSource::insertValue( column.insert(time); break; } - case ValueType::vtDecimal32: [[fallthrough]]; - case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: [[fallthrough]]; + case ValueType::vtDecimal32: + case ValueType::vtDecimal64: + case ValueType::vtDecimal128: case ValueType::vtDecimal256: { auto value = row.get(idx); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index aa38198334e..06b6927bc43 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -296,7 +296,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() switch (global_ctx->chosen_merge_algorithm) { - case MergeAlgorithm::Horizontal : + case MergeAlgorithm::Horizontal: { global_ctx->merging_columns = global_ctx->storage_columns; global_ctx->merging_column_names = global_ctx->all_column_names; @@ -304,7 +304,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->gathering_column_names.clear(); break; } - case MergeAlgorithm::Vertical : + case MergeAlgorithm::Vertical: { ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream(); ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); diff --git a/tests/integration/test_odbc_interaction/test_exiled.py b/tests/integration/test_odbc_interaction/test_exiled.py deleted file mode 100644 index bdc819b8221..00000000000 --- a/tests/integration/test_odbc_interaction/test_exiled.py +++ /dev/null @@ -1,111 +0,0 @@ -import time -import logging -import pytest - -from helpers.cluster import ClickHouseCluster, assert_eq_with_retry -from test_odbc_interaction.test import ( - create_mysql_db, - create_mysql_table, - get_mysql_conn, - skip_test_msan, -) - - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - with_odbc_drivers=True, - main_configs=["configs/openssl.xml", "configs/odbc_logging.xml"], - stay_alive=True, - dictionaries=["configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml"], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] - logging.debug(f"sqlite data received: {sqlite_db}") - node1.exec_in_container( - [ - "sqlite3", - sqlite_db, - "CREATE TABLE t2(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);", - ], - privileged=True, - user="root", - ) - - node1.exec_in_container( - ["sqlite3", sqlite_db, "INSERT INTO t2 values(1, 1, 2, 3);"], - privileged=True, - user="root", - ) - - node1.query("SYSTEM RELOAD DICTIONARY sqlite3_odbc_hashed") - - yield cluster - except Exception as ex: - logging.exception(ex) - raise ex - finally: - cluster.shutdown() - - -# This test kills ClickHouse server and ODBC bridge and in worst scenario -# may cause group test crashes. Thus, this test is executed in a separate "module" -# with separate environment. -def test_bridge_dies_with_parent(started_cluster): - skip_test_msan(node1) - - if node1.is_built_with_address_sanitizer(): - # TODO: Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge in this test and - # that's linked somehow with that we have replaced getauxval() in glibc-compatibility. - # The leak sanitizer calls getauxval() for its own purposes, and our replaced version doesn't seem to be equivalent in that case. - pytest.skip( - "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" - ) - - assert_eq_with_retry( - node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(1))", "3" - ) - - clickhouse_pid = node1.get_process_pid("clickhouse server") - bridge_pid = node1.get_process_pid("odbc-bridge") - assert clickhouse_pid is not None - assert bridge_pid is not None - - try: - node1.exec_in_container( - ["kill", str(clickhouse_pid)], privileged=True, user="root" - ) - except: - pass - - for _ in range(30): - time.sleep(1) - clickhouse_pid = node1.get_process_pid("clickhouse server") - if clickhouse_pid is None: - break - - for _ in range(30): - time.sleep(1) # just for sure, that odbc-bridge caught signal - bridge_pid = node1.get_process_pid("odbc-bridge") - if bridge_pid is None: - break - - if bridge_pid: - out = node1.exec_in_container( - ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], - privileged=True, - user="root", - ) - logging.debug(f"Bridge is running, gdb output:\n{out}") - - try: - assert clickhouse_pid is None - assert bridge_pid is None - finally: - node1.start_clickhouse(20) From 1a2d403f7b7dc16676d1b550dd346c06503c550c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:16:21 +0100 Subject: [PATCH 09/19] Miscellaneae --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 4fd9b4cca45..934639ae604 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -118,7 +118,8 @@ void ODBCSource::insertValue( auto value = row.get(idx); ReadBufferFromString in(value); time_t time = 0; - readDateTimeText(time, in, assert_cast(data_type.get())->getTimeZone()); + const DataTypeDateTime & datetime_type = assert_cast(*data_type); + readDateTimeText(time, in, datetime_type.getTimeZone()); if (time < 0) time = 0; column.insert(static_cast(time)); @@ -129,8 +130,8 @@ void ODBCSource::insertValue( auto value = row.get(idx); ReadBufferFromString in(value); DateTime64 time = 0; - const auto * datetime_type = assert_cast(data_type.get()); - readDateTime64Text(time, datetime_type->getScale(), in, datetime_type->getTimeZone()); + const DataTypeDateTime64 & datetime_type = assert_cast(*data_type); + readDateTime64Text(time, datetime_type.getScale(), in, datetime_type.getTimeZone()); column.insert(time); break; } From 157adefc70d8152f744b23152309cbf03b00c990 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:17:29 +0100 Subject: [PATCH 10/19] Fix error --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 934639ae604..59a5deac960 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -50,7 +50,7 @@ Chunk ODBCSource::generate() const auto & sample = description.sample_block.getByPosition(idx); if (!result.is_null(idx)) - insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); + insertValue(*columns[idx], removeNullable(sample.type), description.types[idx].first, result, idx); else insertDefaultValue(*columns[idx], *sample.column); } From 3b8a8c9649b00d360c5d6169dc769c5371dd44a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:28:30 +0100 Subject: [PATCH 11/19] Better gitignore --- docker/test/stateless/.gitignore | 1 + programs/server/.gitignore | 1 + 2 files changed, 2 insertions(+) create mode 100644 docker/test/stateless/.gitignore diff --git a/docker/test/stateless/.gitignore b/docker/test/stateless/.gitignore new file mode 100644 index 00000000000..928fed26d6d --- /dev/null +++ b/docker/test/stateless/.gitignore @@ -0,0 +1 @@ +/minio_data diff --git a/programs/server/.gitignore b/programs/server/.gitignore index ddc480e4b29..34a774bde9d 100644 --- a/programs/server/.gitignore +++ b/programs/server/.gitignore @@ -2,6 +2,7 @@ /metadata_dropped /data /store +/disks /access /flags /dictionaries_lib From d8e3fa6dc107a4b43edb091a3688b1672a6c2f70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 09:35:57 +0100 Subject: [PATCH 12/19] Fix something strange --- src/Interpreters/fuzzers/execute_query_fuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index 6f84a60f2af..edff202d547 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -3,7 +3,7 @@ #include #include "Processors/Executors/PullingPipelineExecutor.h" -#include +#include #include #include #include From 5c41727725eb1066ff2d86fc5da3e272e03155c1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 3 Mar 2024 14:22:40 +0100 Subject: [PATCH 13/19] http connections pools --- .../Net/include/Poco/Net/HTTPChunkedStream.h | 4 + .../Net/include/Poco/Net/HTTPClientSession.h | 24 +- .../include/Poco/Net/HTTPFixedLengthStream.h | 4 + base/poco/Net/include/Poco/Net/HTTPSession.h | 24 +- base/poco/Net/include/Poco/Net/HTTPStream.h | 2 + base/poco/Net/src/HTTPChunkedStream.cpp | 4 +- base/poco/Net/src/HTTPClientSession.cpp | 23 +- base/poco/Net/src/HTTPFixedLengthStream.cpp | 6 + programs/server/Server.cpp | 18 + .../static-files-disk-uploader.cpp | 6 +- src/Access/Common/AccessType.h | 3 +- src/Access/HTTPAuthClient.h | 2 +- .../CatBoostLibraryBridgeHelper.cpp | 104 +- .../ExternalDictionaryLibraryBridgeHelper.cpp | 38 +- src/BridgeHelper/XDBCBridgeHelper.h | 26 +- src/Client/Connection.cpp | 16 + src/Common/CurrentMetrics.cpp | 12 + src/Common/DNSResolver.cpp | 3 +- src/Common/HTTPConnectionPool.cpp | 865 ++++++++++++++++ src/Common/HTTPConnectionPool.h | 91 ++ src/Common/HostResolvePool.cpp | 293 ++++++ src/Common/HostResolvePool.h | 218 ++++ src/Common/ProfileEvents.cpp | 29 + src/Common/ProxyConfiguration.h | 12 +- src/Common/ProxyListConfigurationResolver.cpp | 2 - .../RemoteProxyConfigurationResolver.cpp | 2 +- src/Common/tests/gtest_connection_pool.cpp | 558 +++++++++++ src/Common/tests/gtest_resolve_pool.cpp | 278 ++++++ src/Core/ServerSettings.h | 11 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 3 +- src/Dictionaries/HTTPDictionarySource.cpp | 91 +- src/Dictionaries/XDBCDictionarySource.cpp | 15 +- src/Dictionaries/XDBCDictionarySource.h | 2 +- src/Disks/IO/ReadBufferFromWebServer.cpp | 81 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 - .../ObjectStorages/Web/WebObjectStorage.cpp | 35 +- src/Functions/sqid.cpp | 2 +- src/IO/ConnectionTimeouts.cpp | 15 + src/IO/ConnectionTimeouts.h | 4 + src/IO/HTTPCommon.cpp | 379 +------ src/IO/HTTPCommon.h | 48 +- src/IO/LimitReadBuffer.cpp | 25 +- src/IO/LimitReadBuffer.h | 18 +- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 +- src/IO/MMapReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadBuffer.h | 12 +- src/IO/ReadBufferFromFileDescriptor.cpp | 4 +- src/IO/ReadBufferFromFileDescriptor.h | 4 +- src/IO/ReadBufferFromIStream.cpp | 40 +- src/IO/ReadBufferFromS3.cpp | 100 +- src/IO/ReadBufferFromS3.h | 6 +- src/IO/ReadSettings.h | 2 +- src/IO/ReadWriteBufferFromHTTP.cpp | 945 +++++++----------- src/IO/ReadWriteBufferFromHTTP.h | 426 ++++---- src/IO/S3/PocoHTTPClient.cpp | 63 +- src/IO/S3/PocoHTTPClient.h | 17 +- src/IO/SeekableReadBuffer.h | 2 +- src/IO/{S3 => }/SessionAwareIOStream.h | 2 +- src/IO/WriteBufferFromHTTP.cpp | 6 +- src/IO/WriteBufferFromHTTP.h | 3 +- src/IO/copyData.cpp | 2 +- src/Interpreters/Context.cpp | 13 - src/Interpreters/Context.h | 4 - src/Interpreters/DDLTask.cpp | 5 + src/Interpreters/InterpreterSystemQuery.cpp | 9 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + .../Formats/Impl/AvroRowInputFormat.cpp | 4 +- src/Storages/MergeTree/DataPartsExchange.cpp | 25 +- src/Storages/MergeTree/DataPartsExchange.h | 10 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 49 +- src/TableFunctions/ITableFunctionXDBC.cpp | 17 +- .../test_cancel_backup.py | 2 +- .../configs/setting.xml | 1 + .../configs/storage_conf.xml | 1 + .../test_disk_over_web_server/test.py | 2 +- tests/integration/test_dns_cache/test.py | 12 +- tests/integration/test_http_failover/test.py | 6 +- .../test_redirect_url_storage/test.py | 2 +- .../test_s3_table_functions/test.py | 1 + .../s3_mocks/unstable_server.py | 2 +- .../0_stateless/00646_url_engine.python | 13 +- .../01271_show_privileges.reference | 1 + .../0_stateless/01293_show_settings.reference | 1 + .../0_stateless/02205_HTTP_user_agent.python | 13 +- .../0_stateless/02233_HTTP_ranged.python | 25 +- ...rofile_events_from_query_log_and_client.sh | 2 +- ...89_reading_from_s3_with_connection_pool.sh | 6 +- .../02833_url_without_path_encoding.sh | 4 +- 92 files changed, 3507 insertions(+), 1771 deletions(-) create mode 100644 src/Common/HTTPConnectionPool.cpp create mode 100644 src/Common/HTTPConnectionPool.h create mode 100644 src/Common/HostResolvePool.cpp create mode 100644 src/Common/HostResolvePool.h create mode 100644 src/Common/tests/gtest_connection_pool.cpp create mode 100644 src/Common/tests/gtest_resolve_pool.cpp rename src/IO/{S3 => }/SessionAwareIOStream.h (97%) diff --git a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h index 5f4729c9278..a6576aa561d 100644 --- a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h @@ -45,6 +45,8 @@ namespace Net ~HTTPChunkedStreamBuf(); void close(); + bool isComplete() const { return _chunk == std::char_traits::eof(); } + protected: int readFromDevice(char * buffer, std::streamsize length); int writeToDevice(const char * buffer, std::streamsize length); @@ -68,6 +70,8 @@ namespace Net ~HTTPChunkedIOS(); HTTPChunkedStreamBuf * rdbuf(); + bool isComplete() const { return _buf.isComplete(); } + protected: HTTPChunkedStreamBuf _buf; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index 7c0caa1c18b..1cef988566c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -210,7 +210,7 @@ namespace Net void setKeepAliveTimeout(const Poco::Timespan & timeout); /// Sets the connection timeout for HTTP connections. - const Poco::Timespan & getKeepAliveTimeout() const; + Poco::Timespan getKeepAliveTimeout() const; /// Returns the connection timeout for HTTP connections. virtual std::ostream & sendRequest(HTTPRequest & request); @@ -275,7 +275,7 @@ namespace Net /// This method should only be called if the request contains /// a "Expect: 100-continue" header. - void flushRequest(); + virtual void flushRequest(); /// Flushes the request stream. /// /// Normally this method does not need to be called. @@ -283,7 +283,7 @@ namespace Net /// fully sent if receiveResponse() is not called, e.g., /// because the underlying socket will be detached. - void reset(); + virtual void reset(); /// Resets the session and closes the socket. /// /// The next request will initiate a new connection, @@ -303,6 +303,9 @@ namespace Net /// Returns true if the proxy should be bypassed /// for the current host. + const Poco::Timestamp & getLastRequest() const; + /// Returns time when connection has been used last time + protected: enum { @@ -338,6 +341,10 @@ namespace Net /// Calls proxyConnect() and attaches the resulting StreamSocket /// to the HTTPClientSession. + void setLastRequest(Poco::Timestamp time); + + void assign(HTTPClientSession & session); + HTTPSessionFactory _proxySessionFactory; /// Factory to create HTTPClientSession to proxy. private: @@ -433,11 +440,20 @@ namespace Net } - inline const Poco::Timespan & HTTPClientSession::getKeepAliveTimeout() const + inline Poco::Timespan HTTPClientSession::getKeepAliveTimeout() const { return _keepAliveTimeout; } + inline const Poco::Timestamp & HTTPClientSession::getLastRequest() const + { + return _lastRequest; + } + + inline void HTTPClientSession::setLastRequest(Poco::Timestamp time) + { + _lastRequest = time; + } } } // namespace Poco::Net diff --git a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h index 2f4df102605..17fa47cfa9b 100644 --- a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h @@ -48,6 +48,8 @@ namespace Net HTTPFixedLengthStreamBuf(HTTPSession & session, ContentLength length, openmode mode); ~HTTPFixedLengthStreamBuf(); + bool isComplete() const; + protected: int readFromDevice(char * buffer, std::streamsize length); int writeToDevice(const char * buffer, std::streamsize length); @@ -67,6 +69,8 @@ namespace Net ~HTTPFixedLengthIOS(); HTTPFixedLengthStreamBuf * rdbuf(); + bool isComplete() const { return _buf.isComplete(); } + protected: HTTPFixedLengthStreamBuf _buf; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index 934b34be5d5..cac14f479db 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -64,6 +64,15 @@ namespace Net Poco::Timespan getTimeout() const; /// Returns the timeout for the HTTP session. + Poco::Timespan getConnectionTimeout() const; + /// Returns connection timeout for the HTTP session. + + Poco::Timespan getSendTimeout() const; + /// Returns send timeout for the HTTP session. + + Poco::Timespan getReceiveTimeout() const; + /// Returns receive timeout for the HTTP session. + bool connected() const; /// Returns true if the underlying socket is connected. @@ -217,12 +226,25 @@ namespace Net return _keepAlive; } - inline Poco::Timespan HTTPSession::getTimeout() const { return _receiveTimeout; } + inline Poco::Timespan HTTPSession::getConnectionTimeout() const + { + return _connectionTimeout; + } + + inline Poco::Timespan HTTPSession::getSendTimeout() const + { + return _sendTimeout; + } + + inline Poco::Timespan HTTPSession::getReceiveTimeout() const + { + return _receiveTimeout; + } inline StreamSocket & HTTPSession::socket() { diff --git a/base/poco/Net/include/Poco/Net/HTTPStream.h b/base/poco/Net/include/Poco/Net/HTTPStream.h index 48502347b2c..a00a861880f 100644 --- a/base/poco/Net/include/Poco/Net/HTTPStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPStream.h @@ -63,6 +63,8 @@ namespace Net ~HTTPIOS(); HTTPStreamBuf * rdbuf(); + bool isComplete() const { return false; } + protected: HTTPStreamBuf _buf; }; diff --git a/base/poco/Net/src/HTTPChunkedStream.cpp b/base/poco/Net/src/HTTPChunkedStream.cpp index 376e3f55492..16ed1e71c31 100644 --- a/base/poco/Net/src/HTTPChunkedStream.cpp +++ b/base/poco/Net/src/HTTPChunkedStream.cpp @@ -49,10 +49,12 @@ HTTPChunkedStreamBuf::~HTTPChunkedStreamBuf() void HTTPChunkedStreamBuf::close() { - if (_mode & std::ios::out) + if (_mode & std::ios::out && _chunk != std::char_traits::eof()) { sync(); _session.write("0\r\n\r\n", 5); + + _chunk = std::char_traits::eof(); } } diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index 2282cca682b..33a3dcc4901 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -227,7 +227,7 @@ void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout) std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) { _pRequestStream = 0; - _pResponseStream = 0; + _pResponseStream = 0; clearException(); _responseReceived = false; @@ -501,5 +501,26 @@ bool HTTPClientSession::bypassProxy() const else return false; } +void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session) +{ + poco_assert (this != &session); + + if (session.buffered()) + throw Poco::LogicException("assign a session with not empty buffered data"); + + if (buffered()) + throw Poco::LogicException("assign to a session with not empty buffered data"); + + attachSocket(session.detachSocket()); + setLastRequest(session.getLastRequest()); + setResolvedHost(session.getResolvedHost()); + setKeepAlive(session.getKeepAlive()); + + setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout()); + setKeepAliveTimeout(session.getKeepAliveTimeout()); + setProxyConfig(session.getProxyConfig()); + + session.reset(); +} } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPFixedLengthStream.cpp b/base/poco/Net/src/HTTPFixedLengthStream.cpp index fd77ff71cd9..837e5723c57 100644 --- a/base/poco/Net/src/HTTPFixedLengthStream.cpp +++ b/base/poco/Net/src/HTTPFixedLengthStream.cpp @@ -43,6 +43,12 @@ HTTPFixedLengthStreamBuf::~HTTPFixedLengthStreamBuf() } +bool HTTPFixedLengthStreamBuf::isComplete() const +{ + return _count == _length; +} + + int HTTPFixedLengthStreamBuf::readFromDevice(char* buffer, std::streamsize length) { int n = 0; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f2c5dcc0f6d..7636f221ab5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include #include @@ -1547,6 +1548,23 @@ try FileCacheFactory::instance().updateSettingsFromConfig(*config); + HTTPConnectionPools::instance().setLimits( + HTTPConnectionPools::Limits{ + new_server_settings.disk_connections_soft_limit, + new_server_settings.disk_connections_warn_limit, + new_server_settings.disk_connections_store_limit, + }, + HTTPConnectionPools::Limits{ + new_server_settings.storage_connections_soft_limit, + new_server_settings.storage_connections_warn_limit, + new_server_settings.storage_connections_store_limit, + }, + HTTPConnectionPools::Limits{ + new_server_settings.http_connections_soft_limit, + new_server_settings.http_connections_warn_limit, + new_server_settings.http_connections_store_limit, + }); + ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. diff --git a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp index d54a2d2da6e..dfe68c819fc 100644 --- a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp +++ b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp @@ -65,7 +65,7 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes /// test mode for integration tests. if (test_mode) - dst_buf = std::make_shared(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); + dst_buf = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); else dst_buf = std::make_shared(dst_file_path); @@ -88,7 +88,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ { dst_path /= "store"; auto files_root = dst_path / prefix; - root_meta = std::make_shared(Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); + root_meta = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); } else { @@ -112,7 +112,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ if (test_mode) { auto files_root = dst_path / prefix; - directory_meta = std::make_shared(Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); + directory_meta = std::make_shared(HTTPConnectionGroupType::HTTP, Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT); } else { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 87f96ca48be..6095f8ce6d3 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -154,7 +154,8 @@ enum class AccessType M(SET_DEFINER, "", USER_NAME, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ - M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MMAP_CACHE, "SYSTEM DROP MMAP, DROP MMAP CACHE, DROP MMAP", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/src/Access/HTTPAuthClient.h b/src/Access/HTTPAuthClient.h index caefe869005..a8b56cf05a7 100644 --- a/src/Access/HTTPAuthClient.h +++ b/src/Access/HTTPAuthClient.h @@ -36,7 +36,7 @@ public: Result authenticateRequest(Poco::Net::HTTPRequest & request) const { - auto session = makeHTTPSession(uri, timeouts); + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts); Poco::Net::HTTPResponse response; auto milliseconds_to_wait = retry_initial_backoff_ms; diff --git a/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp b/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp index 4db1eb720ac..bb9d890a691 100644 --- a/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp +++ b/src/BridgeHelper/CatBoostLibraryBridgeHelper.cpp @@ -58,8 +58,12 @@ bool CatBoostLibraryBridgeHelper::bridgeHandShake() String result; try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials); - readString(result, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(http_timeouts) + .create(credentials); + + readString(result, *buf); } catch (...) { @@ -79,29 +83,29 @@ ExternalModelInfos CatBoostLibraryBridgeHelper::listModels() { startBridgeSync(); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_LIST_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [](std::ostream &) {}, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIST_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .create(credentials); ExternalModelInfos result; UInt64 num_rows; - readIntBinary(num_rows, buf); + readIntBinary(num_rows, *buf); for (UInt64 i = 0; i < num_rows; ++i) { ExternalModelInfo info; - readStringBinary(info.model_path, buf); - readStringBinary(info.model_type, buf); + readStringBinary(info.model_path, *buf); + readStringBinary(info.model_type, *buf); UInt64 t; - readIntBinary(t, buf); + readIntBinary(t, *buf); info.loading_start_time = std::chrono::system_clock::from_time_t(t); - readIntBinary(t, buf); + readIntBinary(t, *buf); info.loading_duration = std::chrono::milliseconds(t); result.push_back(info); @@ -116,17 +120,19 @@ void CatBoostLibraryBridgeHelper::removeModel() assert(model_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_REMOVEMODEL_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this](std::ostream & os) - { - os << "model_path=" << escapeForFileName(*model_path); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEMODEL_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this](std::ostream & os) + { + os << "model_path=" << escapeForFileName(*model_path); + }) + .create(credentials); String result; - readStringBinary(result, buf); + readStringBinary(result, *buf); assert(result == "1"); } @@ -134,14 +140,14 @@ void CatBoostLibraryBridgeHelper::removeAllModels() { startBridgeSync(); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [](std::ostream &){}, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .create(credentials); String result; - readStringBinary(result, buf); + readStringBinary(result, *buf); assert(result == "1"); } @@ -151,18 +157,20 @@ size_t CatBoostLibraryBridgeHelper::getTreeCount() assert(model_path && library_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_GETTREECOUNT_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this](std::ostream & os) - { - os << "library_path=" << escapeForFileName(*library_path) << "&"; - os << "model_path=" << escapeForFileName(*model_path); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_GETTREECOUNT_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this](std::ostream & os) + { + os << "library_path=" << escapeForFileName(*library_path) << "&"; + os << "model_path=" << escapeForFileName(*model_path); + }) + .create(credentials); size_t result; - readIntBinary(result, buf); + readIntBinary(result, *buf); return result; } @@ -177,17 +185,19 @@ ColumnPtr CatBoostLibraryBridgeHelper::evaluate(const ColumnsWithTypeAndName & c assert(model_path); - ReadWriteBufferFromHTTP buf( - createRequestURI(CATBOOST_LIB_EVALUATE_METHOD), - Poco::Net::HTTPRequest::HTTP_POST, - [this, serialized = string_write_buf.str()](std::ostream & os) - { - os << "model_path=" << escapeForFileName(*model_path) << "&"; - os << "data=" << escapeForFileName(serialized); - }, - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIB_EVALUATE_METHOD)) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback( + [this, serialized = string_write_buf.str()](std::ostream & os) + { + os << "model_path=" << escapeForFileName(*model_path) << "&"; + os << "data=" << escapeForFileName(serialized); + }) + .create(credentials); - NativeReader deserializer(buf, /*server_revision*/ 0); + NativeReader deserializer(*buf, /*server_revision*/ 0); Block block_read = deserializer.read(); return block_read.getColumns()[0]; diff --git a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp index fcb8ebd1f22..a37d2bd396f 100644 --- a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp +++ b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp @@ -71,8 +71,12 @@ bool ExternalDictionaryLibraryBridgeHelper::bridgeHandShake() String result; try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials); - readString(result, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(http_timeouts) + .create(credentials); + + readString(result, *buf); } catch (...) { @@ -247,30 +251,28 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadKeys(const Block & requ bool ExternalDictionaryLibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) const { - ReadWriteBufferFromHTTP buf( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - std::move(out_stream_callback), - http_timeouts, credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(http_timeouts) + .withOutCallback(std::move(out_stream_callback)) + .create(credentials); bool res; - readBoolText(res, buf); + readBoolText(res, *buf); return res; } QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) { - auto read_buf_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - std::move(out_stream_callback), - http_timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - getContext()->getReadSettings(), - HTTPHeaderEntries{}); + auto read_buf_ptr = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(http_timeouts) + .withOutCallback(std::move(out_stream_callback)) + .create(credentials); auto source = FormatFactory::instance().getInput(ExternalDictionaryLibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); source->addBuffer(std::move(read_buf_ptr)); diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index d208b8ddab0..838be42357a 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -97,8 +97,12 @@ protected: { try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, getHTTPTimeouts(), credentials); - return checkString(PING_OK_ANSWER, buf); + auto buf = BuilderRWBufferFromHTTP(getPingURI()) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); + + return checkString(PING_OK_ANSWER, *buf); } catch (...) { @@ -198,10 +202,14 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); - bool res; - readBoolText(res, buf); + bool res = false; + readBoolText(res, *buf); is_schema_allowed = res; } @@ -220,10 +228,14 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(getHTTPTimeouts()) + .create(credentials); std::string character; - readStringBinary(character, buf); + readStringBinary(character, *buf); if (character.length() > 1) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Failed to parse quoting style from '{}' for service {}", character, BridgeHelperMixin::serviceAlias()); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 4848f4c7ee5..a11a1243957 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -153,6 +153,12 @@ void Connection::connect(const ConnectionTimeouts & timeouts) current_resolved_address = *it; break; } + catch (DB::NetException &) + { + if (++it == addresses.end()) + throw; + continue; + } catch (Poco::Net::NetException &) { if (++it == addresses.end()) @@ -199,6 +205,16 @@ void Connection::connect(const ConnectionTimeouts & timeouts) LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); } + catch (DB::NetException & e) + { + disconnect(); + + /// Remove this possible stale entry from cache + DNSResolver::instance().removeHostFromCache(host); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); + } catch (Poco::Net::NetException & e) { disconnect(); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82da4c4bbad..f43481f665b 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -275,6 +275,18 @@ M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") + \ + M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ + M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ + \ + M(DiskConnectionsStored, "Total count of sessions stored in the session pool for disks") \ + M(DiskConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for disks") \ + \ + M(HTTPConnectionsStored, "Total count of sessions stored in the session pool for http hosts") \ + M(HTTPConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for http hosts") \ + \ + M(AddressesActive, "Total count of addresses which are used for creation connections with connection pools") \ + #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 5b5f5369d5e..e36e1483da8 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -1,6 +1,7 @@ #include "DNSResolver.h" #include #include +#include #include #include #include @@ -108,7 +109,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host) if (addresses.empty()) { ProfileEvents::increment(ProfileEvents::DNSError); - throw Exception(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host); + throw DB::NetException(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host); } return addresses; diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp new file mode 100644 index 00000000000..18ffef34091 --- /dev/null +++ b/src/Common/HTTPConnectionPool.cpp @@ -0,0 +1,865 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include + +#include "config.h" + +#if USE_SSL +#include +#endif + + +namespace ProfileEvents +{ + extern const Event StorageConnectionsCreated; + extern const Event StorageConnectionsReused; + extern const Event StorageConnectionsReset; + extern const Event StorageConnectionsPreserved; + extern const Event StorageConnectionsExpired; + extern const Event StorageConnectionsErrors; + extern const Event StorageConnectionsElapsedMicroseconds; + + extern const Event DiskConnectionsCreated; + extern const Event DiskConnectionsReused; + extern const Event DiskConnectionsReset; + extern const Event DiskConnectionsPreserved; + extern const Event DiskConnectionsExpired; + extern const Event DiskConnectionsErrors; + extern const Event DiskConnectionsElapsedMicroseconds; + + extern const Event HTTPConnectionsCreated; + extern const Event HTTPConnectionsReused; + extern const Event HTTPConnectionsReset; + extern const Event HTTPConnectionsPreserved; + extern const Event HTTPConnectionsExpired; + extern const Event HTTPConnectionsErrors; + extern const Event HTTPConnectionsElapsedMicroseconds; +} + + +namespace CurrentMetrics +{ + extern const Metric StorageConnectionsStored; + extern const Metric StorageConnectionsTotal; + + extern const Metric DiskConnectionsStored; + extern const Metric DiskConnectionsTotal; + + extern const Metric HTTPConnectionsStored; + extern const Metric HTTPConnectionsTotal; +} + +namespace +{ + Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) + { + Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; + + poco_proxy_config.host = proxy_configuration.host; + poco_proxy_config.port = proxy_configuration.port; + poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); + poco_proxy_config.tunnel = proxy_configuration.tunneling; + poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); + + return poco_proxy_config; + } + + + size_t roundUp(size_t x, size_t rounding) + { + chassert(rounding > 0); + return (x + (rounding - 1)) / rounding * rounding; + } + + + Poco::Timespan divide(const Poco::Timespan span, int divisor) + { + return Poco::Timespan(Poco::Timestamp::TimeDiff(span.totalMicroseconds() / divisor)); + } +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; + extern const int UNSUPPORTED_URI_SCHEME; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForStorageConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::StorageConnectionsCreated, + .reused = ProfileEvents::StorageConnectionsReused, + .reset = ProfileEvents::StorageConnectionsReset, + .preserved = ProfileEvents::StorageConnectionsPreserved, + .expired = ProfileEvents::StorageConnectionsExpired, + .errors = ProfileEvents::StorageConnectionsErrors, + .elapsed_microseconds = ProfileEvents::StorageConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::StorageConnectionsStored, + .active_count = CurrentMetrics::StorageConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForDiskConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::DiskConnectionsCreated, + .reused = ProfileEvents::DiskConnectionsReused, + .reset = ProfileEvents::DiskConnectionsReset, + .preserved = ProfileEvents::DiskConnectionsPreserved, + .expired = ProfileEvents::DiskConnectionsExpired, + .errors = ProfileEvents::DiskConnectionsErrors, + .elapsed_microseconds = ProfileEvents::DiskConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::DiskConnectionsStored, + .active_count = CurrentMetrics::DiskConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getMetricsForHTTPConnectionPool() +{ + return IHTTPConnectionPoolForEndpoint::Metrics{ + .created = ProfileEvents::HTTPConnectionsCreated, + .reused = ProfileEvents::HTTPConnectionsReused, + .reset = ProfileEvents::HTTPConnectionsReset, + .preserved = ProfileEvents::HTTPConnectionsPreserved, + .expired = ProfileEvents::HTTPConnectionsExpired, + .errors = ProfileEvents::HTTPConnectionsErrors, + .elapsed_microseconds = ProfileEvents::HTTPConnectionsElapsedMicroseconds, + .stored_count = CurrentMetrics::HTTPConnectionsStored, + .active_count = CurrentMetrics::HTTPConnectionsTotal, + }; +} + + +IHTTPConnectionPoolForEndpoint::Metrics getConnectionPoolMetrics(HTTPConnectionGroupType type) +{ + switch (type) + { + case HTTPConnectionGroupType::STORAGE: + return getMetricsForStorageConnectionPool(); + case HTTPConnectionGroupType::DISK: + return getMetricsForDiskConnectionPool(); + case HTTPConnectionGroupType::HTTP: + return getMetricsForHTTPConnectionPool(); + } +} + + +class ConnectionGroup +{ +public: + using Ptr = std::shared_ptr; + + explicit ConnectionGroup(HTTPConnectionGroupType type_) : type(type_), metrics(getConnectionPoolMetrics(type_)) { } + + void setLimits(HTTPConnectionPools::Limits limits_) + { + std::lock_guard lock(mutex); + limits = std::move(limits_); + mute_warning_until = 0; + } + + bool isSoftLimitReached() const + { + std::lock_guard lock(mutex); + return total_connections_in_group >= limits.soft_limit; + } + + bool isStoreLimitReached() const + { + std::lock_guard lock(mutex); + return total_connections_in_group >= limits.store_limit; + } + + void atConnectionCreate() + { + std::lock_guard lock(mutex); + + ++total_connections_in_group; + + if (total_connections_in_group >= limits.warning_limit && total_connections_in_group >= mute_warning_until) + { + LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}", type, total_connections_in_group, limits.warning_limit); + mute_warning_until = roundUp(total_connections_in_group, limits.warning_step); + } + } + + void atConnectionDestroy() + { + std::lock_guard lock(mutex); + + --total_connections_in_group; + + const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 10 : 1; + if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit) + { + LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group); + mute_warning_until = 0; + } + } + + HTTPConnectionGroupType getType() const { return type; } + + const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const { return metrics; } + +private: + const HTTPConnectionGroupType type; + const IHTTPConnectionPoolForEndpoint::Metrics metrics; + + LoggerPtr log = getLogger("ConnectionGroup"); + + mutable std::mutex mutex; + HTTPConnectionPools::Limits limits TSA_GUARDED_BY(mutex) = HTTPConnectionPools::Limits(); + size_t total_connections_in_group TSA_GUARDED_BY(mutex) = 0; + size_t mute_warning_until TSA_GUARDED_BY(mutex) = 0; +}; + + +class IExtendedPool : public IHTTPConnectionPoolForEndpoint +{ +public: + using Ptr = std::shared_ptr; + + virtual HTTPConnectionGroupType getGroupType() const = 0; + virtual size_t wipeExpired() = 0; +}; + + +// EndpointConnectionPool manage connections to the endpoint +// Features: +// - it uses HostResolver for address selecting. See Common/HostResolver.h for more info. +// - it minimizes number of `Session::connect()`/`Session::reconnect()` calls +// - stores only connected and ready to use sessions +// - connection could be reused even when limits are reached +// - soft limit, warn limit, store limit +// - `Session::reconnect()` uses the pool as well +// - comprehensive sensors +// - session is reused according its inner state, automatically + + +template +class EndpointConnectionPool : public std::enable_shared_from_this>, public IExtendedPool +{ +private: + friend class HTTPConnectionPools; + + using WeakPtr = std::weak_ptr>; + + class PooledConnection : public Session + { + public: + using Ptr = std::shared_ptr; + + void reconnect() override + { + ProfileEvents::increment(metrics.reset); + Session::close(); + + if (auto lock = pool.lock()) + { + auto timeouts = getTimeouts(*this); + auto new_connection = lock->getConnection(timeouts); + Session::assign(*new_connection); + } + else + { + auto timer = CurrentThread::getProfileEvents().timer(metrics.elapsed_microseconds); + Session::reconnect(); + ProfileEvents::increment(metrics.created); + } + } + + String getTarget() const + { + if (!Session::getProxyConfig().host.empty()) + return fmt::format("{} over proxy {}", Session::getHost(), Session::getProxyConfig().host); + return Session::getHost(); + } + + void flushRequest() override + { + if (bool(request_stream)) + { + request_stream->flush(); + + if (auto * fixed_steam = dynamic_cast(request_stream)) + { + request_stream_completed = fixed_steam->isComplete(); + } + else if (auto * chunked_steam = dynamic_cast(request_stream)) + { + chunked_steam->rdbuf()->close(); + request_stream_completed = chunked_steam->isComplete(); + } + else if (auto * http_stream = dynamic_cast(request_stream)) + { + request_stream_completed = http_stream->isComplete(); + } + else + { + request_stream_completed = false; + } + } + request_stream = nullptr; + + Session::flushRequest(); + } + + std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override + { + std::ostream & result = Session::sendRequest(request); + result.exceptions(std::ios::badbit); + + request_stream = &result; + request_stream_completed = false; + + response_stream = nullptr; + response_stream_completed = true; + + return result; + } + + std::istream & receiveResponse(Poco::Net::HTTPResponse & response) override + { + std::istream & result = Session::receiveResponse(response); + result.exceptions(std::ios::badbit); + + response_stream = &result; + response_stream_completed = false; + + return result; + } + + void reset() override + { + request_stream = nullptr; + request_stream_completed = false; + + response_stream = nullptr; + response_stream_completed = false; + + Session::reset(); + } + + ~PooledConnection() override + { + if (bool(response_stream)) + { + if (auto * fixed_steam = dynamic_cast(response_stream)) + { + response_stream_completed = fixed_steam->isComplete(); + } + else if (auto * chunked_steam = dynamic_cast(response_stream)) + { + response_stream_completed = chunked_steam->isComplete(); + } + else if (auto * http_stream = dynamic_cast(response_stream)) + { + response_stream_completed = http_stream->isComplete(); + } + else + { + response_stream_completed = false; + } + } + response_stream = nullptr; + + if (auto lock = pool.lock()) + lock->atConnectionDestroy(*this); + else + ProfileEvents::increment(metrics.reset); + + CurrentMetrics::sub(metrics.active_count); + } + + private: + friend class EndpointConnectionPool; + + template + explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) + : Session(args...), pool(std::move(pool_)), metrics(std::move(metrics_)) + { + CurrentMetrics::add(metrics.active_count); + } + + template + static Ptr create(Args &&... args) + { + /// Pool is global, we shouldn't attribute this memory to query/user. + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + + struct make_shared_enabler : public PooledConnection + { + explicit make_shared_enabler(Args &&... args) : PooledConnection(std::forward(args)...) { } + }; + return std::make_shared(std::forward(args)...); + } + + void doConnect() + { + Session::reconnect(); + } + + bool isCompleted() const + { + return request_stream_completed && response_stream_completed; + } + + WeakPtr pool; + IHTTPConnectionPoolForEndpoint::Metrics metrics; + + Poco::Logger * log = &Poco::Logger::get("PooledConnection"); + + std::ostream * request_stream = nullptr; + std::istream * response_stream = nullptr; + + bool request_stream_completed = true; + bool response_stream_completed = true; + }; + + using Connection = PooledConnection; + using ConnectionPtr = PooledConnection::Ptr; + + struct GreaterByLastRequest + { + static bool operator()(const ConnectionPtr & l, const ConnectionPtr & r) + { + return l->getLastRequest() + l->getKeepAliveTimeout() > r->getLastRequest() + r->getKeepAliveTimeout(); + } + }; + + using ConnectionsMinHeap = std::priority_queue, GreaterByLastRequest>; + +public: + template + static Ptr create(Args &&... args) + { + struct make_shared_enabler : public EndpointConnectionPool + { + explicit make_shared_enabler(Args &&... args) : EndpointConnectionPool(std::forward(args)...) { } + }; + return std::make_shared(std::forward(args)...); + } + + ~EndpointConnectionPool() override + { + CurrentMetrics::sub(group->getMetrics().stored_count, stored_connections.size()); + } + + String getTarget() const + { + if (!proxy_configuration.isEmpty()) + return fmt::format("{} over proxy {}", host, proxy_configuration.host); + return host; + } + + IHTTPConnectionPoolForEndpoint::ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) override + { + Poco::Timestamp now; + std::vector expired_connections; + + SCOPE_EXIT({ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + expired_connections.clear(); + }); + + { + std::lock_guard lock(mutex); + + wipeExpiredImpl(expired_connections, now); + + if (!stored_connections.empty()) + { + auto it = stored_connections.top(); + stored_connections.pop(); + + setTimeouts(*it, timeouts); + + ProfileEvents::increment(getMetrics().reused, 1); + CurrentMetrics::sub(getMetrics().stored_count, 1); + + return it; + } + } + + return prepareNewConnection(timeouts); + } + + const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const override + { + return group->getMetrics(); + } + + HTTPConnectionGroupType getGroupType() const override + { + return group->getType(); + } + + size_t wipeExpired() override + { + Poco::Timestamp now; + std::vector expired_connections; + + SCOPE_EXIT({ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + expired_connections.clear(); + }); + + std::lock_guard lock(mutex); + return wipeExpiredImpl(expired_connections, now); + } + + size_t wipeExpiredImpl(std::vector & expired_connections, Poco::Timestamp now) TSA_REQUIRES(mutex) + { + while (!stored_connections.empty()) + { + auto connection = stored_connections.top(); + + if (!isExpired(now, connection)) + return stored_connections.size(); + + stored_connections.pop(); + expired_connections.push_back(connection); + } + + CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size()); + ProfileEvents::increment(getMetrics().expired, expired_connections.size()); + + return stored_connections.size(); + } + +private: + EndpointConnectionPool(ConnectionGroup::Ptr group_, String host_, UInt16 port_, bool https_, ProxyConfiguration proxy_configuration_) + : host(std::move(host_)) + , port(port_) + , https(https_) + , proxy_configuration(std::move(proxy_configuration_)) + , group(group_) + { + } + + WeakPtr getWeakFromThis() { return EndpointConnectionPool::weak_from_this(); } + + bool isExpired(Poco::Timestamp & now, ConnectionPtr connection) + { + if (group->isSoftLimitReached()) + return now > (connection->getLastRequest() + divide(connection->getKeepAliveTimeout(), 10)); + return now > connection->getLastRequest() + connection->getKeepAliveTimeout(); + } + + ConnectionPtr allocateNewConnection() + { + ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), getMetrics(), host, port); + connection->setKeepAlive(true); + + if (!proxy_configuration.isEmpty()) + { + connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); + } + + group->atConnectionCreate(); + + return connection; + } + + ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts) + { + auto address = HostResolversPool::instance().getResolver(host)->resolve(); + + auto session = allocateNewConnection(); + + setTimeouts(*session, timeouts); + session->setResolvedHost(*address); + + try + { + auto timer = CurrentThread::getProfileEvents().timer(getMetrics().elapsed_microseconds); + session->doConnect(); + } + catch (...) + { + address.setFail(); + ProfileEvents::increment(getMetrics().errors); + session->reset(); + throw; + } + + ProfileEvents::increment(getMetrics().created); + return session; + } + + void atConnectionDestroy(PooledConnection & connection) + { + group->atConnectionDestroy(); + + if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() + || group->isStoreLimitReached()) + { + ProfileEvents::increment(getMetrics().reset, 1); + return; + } + + auto connection_to_store = allocateNewConnection(); + connection_to_store->assign(connection); + + CurrentMetrics::add(getMetrics().stored_count, 1); + ProfileEvents::increment(getMetrics().preserved, 1); + + { + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + std::lock_guard lock(mutex); + stored_connections.push(connection_to_store); + } + } + + + const std::string host; + const UInt16 port; + const bool https; + const ProxyConfiguration proxy_configuration; + const ConnectionGroup::Ptr group; + + std::mutex mutex; + ConnectionsMinHeap stored_connections TSA_GUARDED_BY(mutex); +}; + +struct EndpointPoolKey +{ + HTTPConnectionGroupType connection_group; + String target_host; + UInt16 target_port; + bool is_target_https; + ProxyConfiguration proxy_config; + + bool operator==(const EndpointPoolKey & rhs) const + { + return std::tie( + connection_group, + target_host, + target_port, + is_target_https, + proxy_config.host, + proxy_config.port, + proxy_config.protocol, + proxy_config.tunneling, + proxy_config.original_request_protocol) + == std::tie( + rhs.connection_group, + rhs.target_host, + rhs.target_port, + rhs.is_target_https, + rhs.proxy_config.host, + rhs.proxy_config.port, + rhs.proxy_config.protocol, + rhs.proxy_config.tunneling, + rhs.proxy_config.original_request_protocol); + } +}; + +struct Hasher +{ + size_t operator()(const EndpointPoolKey & k) const + { + SipHash s; + s.update(k.connection_group); + s.update(k.target_host); + s.update(k.target_port); + s.update(k.is_target_https); + s.update(k.proxy_config.host); + s.update(k.proxy_config.port); + s.update(k.proxy_config.protocol); + s.update(k.proxy_config.tunneling); + s.update(k.proxy_config.original_request_protocol); + return s.get64(); + } +}; + +IExtendedPool::Ptr +createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, bool secure, ProxyConfiguration proxy_configuration) +{ + if (secure) + { +#if USE_SSL + return EndpointConnectionPool::create( + group, std::move(host), port, secure, std::move(proxy_configuration)); +#else + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "Inter-server secret support is disabled, because ClickHouse was built without SSL library"); +#endif + } + else + { + return EndpointConnectionPool::create( + group, std::move(host), port, secure, std::move(proxy_configuration)); + } +} + +class HTTPConnectionPools::Impl +{ +private: + const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 5 * 60; + const Poco::Timespan wipe_timeout = Poco::Timespan(DEFAULT_WIPE_TIMEOUT_SECONDS, 0); + + ConnectionGroup::Ptr disk_group = std::make_shared(HTTPConnectionGroupType::DISK); + ConnectionGroup::Ptr storage_group = std::make_shared(HTTPConnectionGroupType::STORAGE); + ConnectionGroup::Ptr http_group = std::make_shared(HTTPConnectionGroupType::HTTP); + + + /// If multiple mutexes are held simultaneously, + /// they should be locked in this order: + /// HTTPConnectionPools::mutex, then EndpointConnectionPool::mutex, then ConnectionGroup::mutex. + std::mutex mutex; + + std::unordered_map endpoints_pool TSA_GUARDED_BY(mutex); + Poco::Timestamp last_wipe_time TSA_GUARDED_BY(mutex); + +public: + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + Poco::Timestamp now; + + std::lock_guard lock(mutex); + + if (now - last_wipe_time > wipe_timeout) + { + wipeExpired(); + last_wipe_time = now; + } + + return getPoolImpl(type, uri, proxy_configuration); + } + + void setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http) + { + disk_group->setLimits(std::move(disk)); + storage_group->setLimits(std::move(storage)); + http_group->setLimits(std::move(http)); + } + + void dropCache() + { + std::lock_guard lock(mutex); + endpoints_pool.clear(); + } + +protected: + ConnectionGroup::Ptr & getGroup(HTTPConnectionGroupType type) + { + switch (type) + { + case HTTPConnectionGroupType::DISK: + return disk_group; + case HTTPConnectionGroupType::STORAGE: + return storage_group; + case HTTPConnectionGroupType::HTTP: + return http_group; + } + } + + IExtendedPool::Ptr getPoolImpl(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + TSA_REQUIRES(mutex) + { + auto [host, port, secure] = getHostPortSecure(uri, proxy_configuration); + auto key = EndpointPoolKey{type, host, port, secure, proxy_configuration}; + + auto it = endpoints_pool.find(key); + if (it != endpoints_pool.end()) + return it->second; + + it = endpoints_pool.emplace(key, createConnectionPool(getGroup(type), std::move(host), port, secure, proxy_configuration)).first; + + return it->second; + } + + void wipeExpired() TSA_REQUIRES(mutex) + { + std::vector keys_to_drop; + + for (auto & [key, pool] : endpoints_pool) + { + auto left_connections = pool->wipeExpired(); + if (left_connections == 0 && pool->getGroupType() != HTTPConnectionGroupType::DISK) + keys_to_drop.push_back(key); + } + + for (const auto & key : keys_to_drop) + endpoints_pool.erase(key); + } + + static bool useSecureConnection(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + if (uri.getScheme() == "http") + return false; + + if (uri.getScheme() != "https") + throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); + + if (!proxy_configuration.isEmpty()) + { + if (ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol && !proxy_configuration.tunneling) + { + // If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request. + // The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server. + // There is a weak link in the security, but that's what the user opted for. + return false; + } + } + + return true; + } + + static std::tuple getHostPortSecure(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + { + return std::make_tuple(uri.getHost(), uri.getPort(), useSecureConnection(uri, proxy_configuration)); + } +}; + +HTTPConnectionPools::HTTPConnectionPools() + : impl(std::make_unique()) +{ +} + +HTTPConnectionPools & HTTPConnectionPools::instance() +{ + static HTTPConnectionPools instance; + return instance; +} + +void HTTPConnectionPools::setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http) +{ + impl->setLimits(std::move(disk), std::move(storage), std::move(http)); +} + +void HTTPConnectionPools::dropCache() +{ + impl->dropCache(); +} + +IHTTPConnectionPoolForEndpoint::Ptr +HTTPConnectionPools::getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) +{ + return impl->getPool(type, uri, proxy_configuration); +} +} diff --git a/src/Common/HTTPConnectionPool.h b/src/Common/HTTPConnectionPool.h new file mode 100644 index 00000000000..7fd0847f5a7 --- /dev/null +++ b/src/Common/HTTPConnectionPool.h @@ -0,0 +1,91 @@ +#pragma once + +#include + +#include +#include +#include +#include + +#include + +#include +#include + +#include +#include + +namespace DB +{ + +class IHTTPConnectionPoolForEndpoint +{ +public: + struct Metrics + { + const ProfileEvents::Event created = ProfileEvents::end(); + const ProfileEvents::Event reused = ProfileEvents::end(); + const ProfileEvents::Event reset = ProfileEvents::end(); + const ProfileEvents::Event preserved = ProfileEvents::end(); + const ProfileEvents::Event expired = ProfileEvents::end(); + const ProfileEvents::Event errors = ProfileEvents::end(); + const ProfileEvents::Event elapsed_microseconds = ProfileEvents::end(); + + const CurrentMetrics::Metric stored_count = CurrentMetrics::end(); + const CurrentMetrics::Metric active_count = CurrentMetrics::end(); + }; + + using Ptr = std::shared_ptr; + using Connection = Poco::Net::HTTPClientSession; + using ConnectionPtr = std::shared_ptr; + + /// can throw Poco::Net::Exception, DB::NetException, DB::Exception + virtual ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) = 0; + virtual const Metrics & getMetrics() const = 0; + virtual ~IHTTPConnectionPoolForEndpoint() = default; + +protected: + IHTTPConnectionPoolForEndpoint() = default; + + IHTTPConnectionPoolForEndpoint(const IHTTPConnectionPoolForEndpoint &) = delete; + IHTTPConnectionPoolForEndpoint & operator=(const IHTTPConnectionPoolForEndpoint &) = delete; +}; + +enum class HTTPConnectionGroupType +{ + DISK, + STORAGE, + HTTP, +}; + +class HTTPConnectionPools +{ +public: + struct Limits + { + size_t soft_limit = 100; + size_t warning_limit = 1000; + size_t store_limit = 10000; + + static constexpr size_t warning_step = 100; + }; + +private: + HTTPConnectionPools(); + HTTPConnectionPools(const HTTPConnectionPools &) = delete; + HTTPConnectionPools & operator=(const HTTPConnectionPools &) = delete; + +public: + static HTTPConnectionPools & instance(); + + void setLimits(Limits disk, Limits storage, Limits http); + void dropCache(); + + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration); + +private: + class Impl; + std::unique_ptr impl; +}; + +} diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp new file mode 100644 index 00000000000..f6cc9c919ba --- /dev/null +++ b/src/Common/HostResolvePool.cpp @@ -0,0 +1,293 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace ProfileEvents +{ + extern const Event AddressesDiscovered; + extern const Event AddressesExpired; + extern const Event AddressesFailScored; +} + +namespace CurrentMetrics +{ + extern const Metric AddressesActive; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DNS_ERROR; +} + +HostResolverMetrics HostResolver::getMetrics() +{ + return HostResolverMetrics{ + .discovered = ProfileEvents::AddressesDiscovered, + .expired = ProfileEvents::AddressesExpired, + .failed = ProfileEvents::AddressesFailScored, + .active_count = CurrentMetrics::AddressesActive, + }; +} + +HostResolver::WeakPtr HostResolver::getWeakFromThis() +{ + return weak_from_this(); +} + +HostResolver::HostResolver(String host_, Poco::Timespan history_) + : host(std::move(host_)) + , history(history_) + , resolve_function([](const String & host_to_resolve) { return DNSResolver::instance().resolveHostAll(host_to_resolve); }) +{ + update(); +} + +HostResolver::HostResolver( + ResolveFunction && resolve_function_, String host_, Poco::Timespan history_) + : host(std::move(host_)), history(history_), resolve_function(std::move(resolve_function_)) +{ + update(); +} + +HostResolver::~HostResolver() +{ + std::lock_guard lock(mutex); + CurrentMetrics::sub(metrics.active_count, records.size()); + records.clear(); +} + +void HostResolver::Entry::setFail() +{ + fail = true; + + if (auto lock = pool.lock()) + lock->setFail(address); +} + +HostResolver::Entry::~Entry() +{ + if (!fail) + { + if (auto lock = pool.lock()) + lock->setSuccess(address); + } +} + +void HostResolver::update() +{ + MemoryTrackerSwitcher switcher{&total_memory_tracker}; + + auto next_gen = resolve_function(host); + if (next_gen.empty()) + throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host); + + std::sort(next_gen.begin(), next_gen.end()); + + Poco::Timestamp now; + + std::lock_guard lock(mutex); + updateImpl(now, next_gen); +} + +void HostResolver::reset() +{ + std::lock_guard lock(mutex); + + CurrentMetrics::sub(metrics.active_count, records.size()); + records.clear(); +} + +void HostResolver::updateWeights() +{ + updateWeightsImpl(); + + if (getTotalWeight() == 0 && !records.empty()) + { + for (auto & rec : records) + rec.failed = false; + + updateWeightsImpl(); + } + + chassert((getTotalWeight() > 0 && !records.empty()) || records.empty()); + random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); +} + +HostResolver::Entry HostResolver::resolve() +{ + if (isUpdateNeeded()) + update(); + + std::lock_guard lock(mutex); + return Entry(*this, selectBest()); +} + +void HostResolver::setSuccess(const Poco::Net::IPAddress & address) +{ + std::lock_guard lock(mutex); + + auto it = find(address); + if (it == records.end()) + return; + + auto old_weight = it->getWeight(); + ++it->usage; + auto new_weight = it->getWeight(); + + if (old_weight != new_weight) + updateWeights(); +} + +void HostResolver::setFail(const Poco::Net::IPAddress & address) +{ + Poco::Timestamp now; + + { + std::lock_guard lock(mutex); + + auto it = find(address); + if (it == records.end()) + return; + + it->failed = true; + it->fail_time = now; + } + + ProfileEvents::increment(metrics.failed); + update(); +} + +Poco::Net::IPAddress HostResolver::selectBest() +{ + chassert(!records.empty()); + size_t weight = random_weight_picker(thread_local_rng); + auto it = std::partition_point(records.begin(), records.end(), [&](const Record & rec) { return rec.weight_prefix_sum <= weight; }); + chassert(it != records.end()); + return it->address; +} + +HostResolver::Records::iterator HostResolver::find(const Poco::Net::IPAddress & addr) TSA_REQUIRES(mutex) +{ + return std::lower_bound( + records.begin(), records.end(), addr, [](const Record & rec, const Poco::Net::IPAddress & value) { return rec.address < value; }); +} + +bool HostResolver::isUpdateNeeded() +{ + Poco::Timestamp now; + + std::lock_guard lock(mutex); + return last_resolve_time + history < now || records.empty(); +} + +void HostResolver::updateImpl(Poco::Timestamp now, std::vector & next_gen) + TSA_REQUIRES(mutex) +{ + const auto last_effective_resolve = now - history; + + Records merged; + merged.reserve(records.size() + next_gen.size()); + + auto it_before = records.begin(); + auto it_next = next_gen.begin(); + + while (it_before != records.end() || it_next != next_gen.end()) + { + if (it_next == next_gen.end() || (it_before != records.end() && it_before->address < *it_next)) + { + if (it_before->resolve_time >= last_effective_resolve) + merged.push_back(*it_before); + else + { + CurrentMetrics::sub(metrics.active_count, 1); + ProfileEvents::increment(metrics.expired, 1); + } + ++it_before; + } + else if (it_before == records.end() || (it_next != next_gen.end() && *it_next < it_before->address)) + { + CurrentMetrics::add(metrics.active_count, 1); + ProfileEvents::increment(metrics.discovered, 1); + merged.push_back(Record(*it_next, now)); + ++it_next; + } + else + { + merged.push_back(*it_before); + merged.back().resolve_time = now; + + ++it_before; + ++it_next; + } + } + + for (auto & rec : merged) + if (rec.failed && rec.fail_time < last_effective_resolve) + rec.failed = false; + + chassert(std::is_sorted(merged.begin(), merged.end())); + + last_resolve_time = now; + records.swap(merged); + + if (records.empty()) + throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host); + + updateWeights(); +} + +size_t HostResolver::getTotalWeight() const +{ + if (records.empty()) + return 0; + return records.back().weight_prefix_sum; +} + + +void HostResolver::updateWeightsImpl() +{ + size_t total_weight_next = 0; + + for (auto & rec: records) + { + total_weight_next += rec.getWeight(); + rec.weight_prefix_sum = total_weight_next; + } +} + +HostResolversPool & HostResolversPool::instance() +{ + static HostResolversPool instance; + return instance; +} + +void HostResolversPool::dropCache() +{ + std::lock_guard lock(mutex); + host_pools.clear(); +} + +HostResolver::Ptr HostResolversPool::getResolver(const String & host) +{ + std::lock_guard lock(mutex); + + auto it = host_pools.find(host); + if (it != host_pools.end()) + return it->second; + + it = host_pools.emplace(host, HostResolver::create(host)).first; + + return it->second; +} + +} diff --git a/src/Common/HostResolvePool.h b/src/Common/HostResolvePool.h new file mode 100644 index 00000000000..2a31cec3b2d --- /dev/null +++ b/src/Common/HostResolvePool.h @@ -0,0 +1,218 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include + +#include +#include + +// That class resolves host into multiply addresses +// Features: +// - balance address usage. +// `selectBest()` chooses the address by random with weights. +// The more ip is used the lesser weight it has. When new address is happened, it takes more weight. +// But still not all requests are assigned to the new address. +// - join resolve results +// In case when host is resolved into different set of addresses, this class join all that addresses and use them. +// An address expires after `history_` time. +// - failed address pessimization +// If an address marked with `setFail()` it is marked as faulty. Such address won't be selected until either +// a) it still occurs in resolve set after `history_` time or b) all other addresses are pessimized as well. +// - resolve schedule +// Addresses are resolved through `DB::DNSResolver::instance()`. +// Usually it does not happen more often than once in `history_` time. +// But also new resolve performed each `setFail()` call. + +namespace DB +{ + +struct HostResolverMetrics +{ + const ProfileEvents::Event discovered = ProfileEvents::end(); + const ProfileEvents::Event expired = ProfileEvents::end(); + const ProfileEvents::Event failed = ProfileEvents::end(); + + const CurrentMetrics::Metric active_count = CurrentMetrics::end(); +}; + +constexpr size_t DEFAULT_RESOLVE_TIME_HISTORY_SECONDS = 2*60; + + +class HostResolver : public std::enable_shared_from_this +{ +private: + using WeakPtr = std::weak_ptr; + +public: + using Ptr = std::shared_ptr; + + template + static Ptr create(Args&&... args) + { + struct make_shared_enabler : public HostResolver + { + explicit make_shared_enabler(Args&&... args) : HostResolver(std::forward(args)...) {} + }; + return std::make_shared(std::forward(args)...); + } + + virtual ~HostResolver(); + + class Entry + { + public: + explicit Entry(Entry && entry) = default; + explicit Entry(Entry & entry) = delete; + + // no access as r-value + const String * operator->() && = delete; + const String * operator->() const && = delete; + const String & operator*() && = delete; + const String & operator*() const && = delete; + + const String * operator->() & { return &resolved_host; } + const String * operator->() const & { return &resolved_host; } + const String & operator*() & { return resolved_host; } + const String & operator*() const & { return resolved_host; } + + void setFail(); + ~Entry(); + + private: + friend class HostResolver; + + Entry(HostResolver & pool_, Poco::Net::IPAddress address_) + : pool(pool_.getWeakFromThis()) + , address(std::move(address_)) + , resolved_host(address.toString()) + { } + + HostResolver::WeakPtr pool; + const Poco::Net::IPAddress address; + const String resolved_host; + + bool fail = false; + }; + + /// can throw NetException(ErrorCodes::DNS_ERROR, ...), Exception(ErrorCodes::BAD_ARGUMENTS, ...) + Entry resolve(); + void update(); + void reset(); + + static HostResolverMetrics getMetrics(); + +protected: + explicit HostResolver( + String host_, + Poco::Timespan history_ = Poco::Timespan(DEFAULT_RESOLVE_TIME_HISTORY_SECONDS, 0)); + + using ResolveFunction = std::function (const String & host)>; + HostResolver(ResolveFunction && resolve_function_, + String host_, + Poco::Timespan history_); + + friend class Entry; + WeakPtr getWeakFromThis(); + + void setSuccess(const Poco::Net::IPAddress & address); + void setFail(const Poco::Net::IPAddress & address); + + struct Record + { + Record(Poco::Net::IPAddress address_, Poco::Timestamp resolve_time_) + : address(std::move(address_)) + , resolve_time(resolve_time_) + {} + + explicit Record(Record && rec) = default; + Record& operator=(Record && s) = default; + + explicit Record(const Record & rec) = default; + Record& operator=(const Record & s) = default; + + Poco::Net::IPAddress address; + Poco::Timestamp resolve_time; + size_t usage = 0; + bool failed = false; + Poco::Timestamp fail_time = 0; + + size_t weight_prefix_sum; + + bool operator <(const Record & r) const + { + return address < r.address; + } + + size_t getWeight() const + { + if (failed) + return 0; + + /// There is no goal to make usage's distribution ideally even + /// The goal is to chose more often new address, but still use old addresses as well + /// when all addresses have usage counter greater than 10000, + /// no more corrections are needed, just random choice is ok + if (usage > 10000) + return 1; + if (usage > 1000) + return 5; + if (usage > 100) + return 8; + return 10; + } + }; + + using Records = std::vector; + + Poco::Net::IPAddress selectBest() TSA_REQUIRES(mutex); + Records::iterator find(const Poco::Net::IPAddress & address) TSA_REQUIRES(mutex); + bool isUpdateNeeded(); + + void updateImpl(Poco::Timestamp now, std::vector & next_gen) TSA_REQUIRES(mutex); + void updateWeights() TSA_REQUIRES(mutex); + void updateWeightsImpl() TSA_REQUIRES(mutex); + size_t getTotalWeight() const TSA_REQUIRES(mutex); + + const String host; + const Poco::Timespan history; + const HostResolverMetrics metrics = getMetrics(); + + // for tests purpose + const ResolveFunction resolve_function; + + std::mutex mutex; + + Poco::Timestamp last_resolve_time TSA_GUARDED_BY(mutex); + Records records TSA_GUARDED_BY(mutex); + + std::uniform_int_distribution random_weight_picker TSA_GUARDED_BY(mutex); + + Poco::Logger * log = &Poco::Logger::get("ConnectionPool"); +}; + +class HostResolversPool +{ +private: + HostResolversPool() = default; + HostResolversPool(const HostResolversPool &) = delete; + HostResolversPool & operator=(const HostResolversPool &) = delete; + +public: + static HostResolversPool & instance(); + + void dropCache(); + + HostResolver::Ptr getResolver(const String & host); +private: + std::mutex mutex; + std::unordered_map host_pools TSA_GUARDED_BY(mutex); +}; + +} + diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 052c059a72d..0c9582ab4fb 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -696,6 +696,35 @@ The server successfully detected this situation and will download merged part fr \ M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \ M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas") \ + \ + M(StorageConnectionsCreated, "Number of created connections for storages") \ + M(StorageConnectionsReused, "Number of reused connections for storages") \ + M(StorageConnectionsReset, "Number of reset connections for storages") \ + M(StorageConnectionsPreserved, "Number of preserved connections for storages") \ + M(StorageConnectionsExpired, "Number of expired connections for storages") \ + M(StorageConnectionsErrors, "Number of cases when creation of a connection for storage is failed") \ + M(StorageConnectionsElapsedMicroseconds, "Total time spend on creating connections for storages") \ + \ + M(DiskConnectionsCreated, "Number of created connections for disk") \ + M(DiskConnectionsReused, "Number of reused connections for disk") \ + M(DiskConnectionsReset, "Number of reset connections for disk") \ + M(DiskConnectionsPreserved, "Number of preserved connections for disk") \ + M(DiskConnectionsExpired, "Number of expired connections for disk") \ + M(DiskConnectionsErrors, "Number of cases when creation of a connection for disk is failed") \ + M(DiskConnectionsElapsedMicroseconds, "Total time spend on creating connections for disk") \ + \ + M(HTTPConnectionsCreated, "Number of created http connections") \ + M(HTTPConnectionsReused, "Number of reused http connections") \ + M(HTTPConnectionsReset, "Number of reset http connections") \ + M(HTTPConnectionsPreserved, "Number of preserved http connections") \ + M(HTTPConnectionsExpired, "Number of expired http connections") \ + M(HTTPConnectionsErrors, "Number of cases when creation of a http connection failed") \ + M(HTTPConnectionsElapsedMicroseconds, "Total time spend on creating http connections") \ + \ + M(AddressesDiscovered, "Total count of new addresses in dns resolve results for connection pools") \ + M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for for connection pools") \ + M(AddressesFailScored, "Total count of new addresses in dns resolve results for for connection pools") \ + #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index 53e569bf6e4..11a09cb5924 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -44,11 +44,13 @@ struct ProxyConfiguration } } - std::string host; - Protocol protocol; - uint16_t port; - bool tunneling; - Protocol original_request_protocol; + std::string host = std::string{}; + Protocol protocol = Protocol::HTTP; + uint16_t port = 0; + bool tunneling = false; + Protocol original_request_protocol = Protocol::HTTP; + + bool isEmpty() const { return host.size() == 0; } }; } diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index 01a6f52185f..c9b8923929a 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -26,8 +26,6 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve() auto & proxy = proxies[index]; - LOG_DEBUG(getLogger("ProxyListConfigurationResolver"), "Use proxy: {}", proxies[index].toString()); - return ProxyConfiguration { proxy.getHost(), ProxyConfiguration::protocolFromString(proxy.getScheme()), diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index 117c8a34dbb..ef972a8e318 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -69,7 +69,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve() { auto resolved_endpoint = endpoint; resolved_endpoint.setHost(resolved_hosts[i].toString()); - session = makeHTTPSession(resolved_endpoint, timeouts); + session = makeHTTPSession(HTTPConnectionGroupType::HTTP, resolved_endpoint, timeouts); try { diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp new file mode 100644 index 00000000000..01b78958442 --- /dev/null +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -0,0 +1,558 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace +{ + +size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits::max()) +{ + const size_t buffer_size = 4096; + char buffer[buffer_size]; + + size_t total_read = 0; + + while (count > buffer_size) + { + in.read(buffer, buffer_size); + size_t read = in.gcount(); + out.write(buffer, read); + count -= read; + total_read += read; + + if (read == 0) + return total_read; + } + + in.read(buffer, count); + size_t read = in.gcount(); + out.write(buffer, read); + total_read += read; + + return total_read; +} + +class MockRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit MockRequestHandler(std::shared_ptr> slowdown_) + : slowdown(std::move(slowdown_)) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override + { + response.setStatus(Poco::Net::HTTPResponse::HTTP_OK); + auto size = request.getContentLength(); + if (size > 0) + response.setContentLength(size); // ContentLength is required for keep alive + else + response.setChunkedTransferEncoding(true); // or chunk encoding + + sleepForSeconds(*slowdown); + + stream_copy_n(request.stream(), response.send(), size); + } + + std::shared_ptr> slowdown; +}; + +class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + explicit HTTPRequestHandlerFactory(std::shared_ptr> slowdown_) + : slowdown(std::move(slowdown_)) + { + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override + { + return new MockRequestHandler(slowdown); + } + + std::shared_ptr> slowdown; +}; + +} + +using HTTPSession = Poco::Net::HTTPClientSession; +using HTTPSessionPtr = std::shared_ptr; + +class ConnectionPoolTest : public testing::Test { +protected: + ConnectionPoolTest() + { + startServer(); + } + + void SetUp() override { + timeouts = DB::ConnectionTimeouts(); + DB::HTTPConnectionPools::Limits def_limits{}; + DB::HTTPConnectionPools::instance().setLimits(def_limits, def_limits, def_limits); + + setSlowDown(0); + + DB::HTTPConnectionPools::instance().dropCache(); + DB::CurrentThread::getProfileEvents().reset(); + // Code here will be called immediately after the constructor (right + // before each test). + } + + void TearDown() override { + // Code here will be called immediately after each test (right + // before the destructor). + } + + DB::IHTTPConnectionPoolForEndpoint::Ptr getPool() + { + auto uri = Poco::URI(getServerUrl()); + return DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{}); + } + + std::string getServerUrl() const + { + return "http://" + server_data.socket->address().toString(); + } + + void startServer() + { + server_data.reset(); + server_data.params = new Poco::Net::HTTPServerParams(); + server_data.socket = std::make_unique(server_data.port); + server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive); + server_data.server = std::make_unique( + server_data.handler_factory, *server_data.socket, server_data.params); + + server_data.server->start(); + } + + Poco::Net::HTTPServer & getServer() const + { + return *server_data.server; + } + + void setSlowDown(size_t seconds) + { + *slowdown_receive = seconds; + } + + DB::ConnectionTimeouts timeouts; + std::shared_ptr> slowdown_receive = std::make_shared>(0); + + struct ServerData + { + // just some port to avoid collisions with others tests + UInt16 port = 9871; + Poco::Net::HTTPServerParams::Ptr params; + std::unique_ptr socket; + HTTPRequestHandlerFactory::Ptr handler_factory; + std::unique_ptr server; + + ServerData() = default; + ServerData(ServerData &&) = default; + ServerData & operator =(ServerData &&) = delete; + + void reset() + { + if (server) + server->stop(); + + server = nullptr; + handler_factory = nullptr; + socket = nullptr; + params = nullptr; + } + + ~ServerData() { + reset(); + } + }; + + ServerData server_data; +}; + + +void wait_until(std::function pred) +{ + while (!pred()) + sleepForMilliseconds(250); +} + +void echoRequest(String data, HTTPSession & session) +{ + { + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_PUT, "/", "HTTP/1.1"); // HTTP/1.1 is required for keep alive + request.setContentLength(data.size()); + std::ostream & ostream = session.sendRequest(request); + ostream << data; + } + + { + std::stringstream result; + Poco::Net::HTTPResponse response; + std::istream & istream = session.receiveResponse(response); + ASSERT_EQ(response.getStatus(), Poco::Net::HTTPResponse::HTTP_OK); + + stream_copy_n(istream, result); + ASSERT_EQ(data, result.str()); + } +} + +TEST_F(ConnectionPoolTest, CanConnect) +{ + auto pool = getPool(); + auto connection = pool->getConnection(timeouts); + + ASSERT_TRUE(connection->connected()); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); +} + +TEST_F(ConnectionPoolTest, CanRequest) +{ + auto pool = getPool(); + auto connection = pool->getConnection(timeouts); + + echoRequest("Hello", *connection); + + ASSERT_EQ(1, getServer().totalConnections()); + ASSERT_EQ(1, getServer().currentConnections()); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); +} + +TEST_F(ConnectionPoolTest, CanPreserve) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + } + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + } + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); + + { + auto connection = pool->getConnection(timeouts); + + ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + + echoRequest("Hello", *connection); + + ASSERT_EQ(1, getServer().totalConnections()); + ASSERT_EQ(1, getServer().currentConnections()); + + connection->reset(); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse10) +{ + auto pool = getPool(); + + + for (int i = 0; i < 10; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + + { + auto connection = pool->getConnection(timeouts); + connection->reset(); // reset just not to wait its expiration here + } + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(1, getServer().totalConnections()); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReuse5) +{ + timeouts.withHTTPKeepAliveTimeout(1); + + auto pool = getPool(); + + std::vector connections; + connections.reserve(5); + for (int i = 0; i < 5; ++i) + { + connections.push_back(pool->getConnection(timeouts)); + } + connections.clear(); + + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); + + wait_until([&] () { return getServer().currentConnections() == 5; }); + ASSERT_EQ(5, getServer().currentConnections()); + ASSERT_EQ(5, getServer().totalConnections()); + + for (int i = 0; i < 5; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + + ASSERT_EQ(5, getServer().totalConnections()); + + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); +} + +TEST_F(ConnectionPoolTest, CanReconnectAndCreate) +{ + auto pool = getPool(); + + std::vector in_use; + + const size_t count = 2; + for (int i = 0; i < count; ++i) + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + in_use.push_back(connection); + } + + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + + ASSERT_EQ(count, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + auto connection = std::move(in_use.back()); + in_use.pop_back(); + + echoRequest("Hello", *connection); + + connection->abort(); // further usage requires reconnect, new connection + + echoRequest("Hello", *connection); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 1; }); + ASSERT_EQ(1, getServer().currentConnections()); + ASSERT_EQ(count+1, getServer().totalConnections()); + + ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, CanReconnectAndReuse) +{ + auto pool = getPool(); + + std::vector in_use; + + const size_t count = 2; + for (int i = 0; i < count; ++i) + { + auto connection = pool->getConnection(timeouts); + // DB::setReuseTag(*connection); + in_use.push_back(std::move(connection)); + } + + auto connection = std::move(in_use.back()); + in_use.pop_back(); + in_use.clear(); // other connection will be reused + + echoRequest("Hello", *connection); + + connection->abort(); // further usage requires reconnect, reuse connection from pool + + echoRequest("Hello", *connection); + + connection->reset(); + + wait_until([&] () { return getServer().currentConnections() == 0; }); + ASSERT_EQ(0, getServer().currentConnections()); + ASSERT_EQ(2, getServer().totalConnections()); + + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); +} + +TEST_F(ConnectionPoolTest, ReceiveTimeout) +{ + setSlowDown(2); + timeouts.withReceiveTimeout(1); + + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + ASSERT_ANY_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + + { + timeouts.withReceiveTimeout(3); + auto connection = pool->getConnection(timeouts); + ASSERT_NO_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + + { + /// timeouts have effect for reused session + timeouts.withReceiveTimeout(1); + auto connection = pool->getConnection(timeouts); + ASSERT_ANY_THROW( + echoRequest("Hello", *connection); + ); + } + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} + +TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP) +{ + std::string_view message = "Hello ReadWriteBufferFromHTTP"; + auto uri = Poco::URI(getServerUrl()); + auto metrics = DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{})->getMetrics(); + Poco::Net::HTTPBasicCredentials empty_creds; + auto buf_from_http = DB::BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withOutCallback( + [&] (std::ostream & in) + { + in << message; + }) + .withDelayInit(false) + .create(empty_creds); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); + + char buf[256]; + std::fill(buf, buf + sizeof(buf), 0); + + buf_from_http->readStrict(buf, message.size()); + ASSERT_EQ(std::string_view(buf), message); + ASSERT_TRUE(buf_from_http->eof()); + + buf_from_http.reset(); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); +} + +TEST_F(ConnectionPoolTest, HardLimit) +{ + DB::HTTPConnectionPools::Limits zero_limits {0, 0, 0}; + DB::HTTPConnectionPools::instance().setLimits(zero_limits, zero_limits, zero_limits); + + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} diff --git a/src/Common/tests/gtest_resolve_pool.cpp b/src/Common/tests/gtest_resolve_pool.cpp new file mode 100644 index 00000000000..25e867fdebc --- /dev/null +++ b/src/Common/tests/gtest_resolve_pool.cpp @@ -0,0 +1,278 @@ +#include +#include +#include + +#include +#include + +class ResolvePoolMock : public DB::HostResolver +{ +public: + using ResolveFunction = DB::HostResolver::ResolveFunction; + + ResolvePoolMock(String host_, Poco::Timespan history_, ResolveFunction && func) + : DB::HostResolver(std::move(func), std::move(host_), history_) + { + } +}; + +class ResolvePoolTest : public testing::Test +{ +protected: + ResolvePoolTest() + { + DB::HostResolversPool::instance().dropCache(); + } + + void SetUp() override { + DB::CurrentThread::getProfileEvents().reset(); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + + addresses = std::set{"127.0.0.1", "127.0.0.2", "127.0.0.3"}; + // Code here will be called immediately after the constructor (right + // before each test). + } + + void TearDown() override { + // Code here will be called immediately after each test (right + // before the destructor). + } + + DB::HostResolver::Ptr make_resolver(size_t history_ms = 200) + { + auto resolve_func = [&] (const String &) + { + std::vector result; + result.reserve(addresses.size()); + for (const auto & item : addresses) + { + result.push_back(Poco::Net::IPAddress(item)); + } + return result; + }; + + + return std::make_shared("some_host", Poco::Timespan(history_ms * 1000), std::move(resolve_func)); + } + + DB::HostResolverMetrics metrics = DB::HostResolver::getMetrics(); + std::set addresses; +}; + +TEST_F(ResolvePoolTest, CanResolve) +{ + auto resolver = make_resolver(); + auto address = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*address)); + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count)); +} + +TEST_F(ResolvePoolTest, CanResolveAll) +{ + auto resolver = make_resolver(); + + std::set results; + while (results.size() != addresses.size()) + { + auto next_addr = resolver->resolve(); + results.insert(*next_addr); + } + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); +} + +size_t getSum(std::map container) +{ + size_t sum = 0; + for (auto & [_, val] : container) + { + sum += val; + } + return sum; +} + +size_t getMin(std::map container) +{ + if (container.empty()) + return 0; + + size_t min_val = container.begin()->second; + for (auto & [_, val] : container) + { + min_val = std::min(min_val, val); + } + return min_val; +} + +double getMean(std::map container) +{ + return 1.0 * getSum(container) / container.size(); +} + +double getMaxDiff(std::map container, double ref_val) +{ + double diff = 0.0; + for (auto & [_, val] : container) + { + diff = std::max(std::fabs(val - ref_val), diff); + } + + return diff; +} + +TEST_F(ResolvePoolTest, CanResolveEvenly) +{ + auto resolver = make_resolver(); + + std::map results; + + for (size_t i = 0; i < 50000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + auto mean = getMean(results); + auto diff = getMaxDiff(results, mean); + + ASSERT_GT(0.3 * mean, diff); +} + +TEST_F(ResolvePoolTest, CanMerge) +{ + auto resolver = make_resolver(100000); + auto address = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*address)); + + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + + auto old_addresses = addresses; + addresses = std::set{"127.0.0.4", "127.0.0.5"}; + + + resolver->update(); + ASSERT_EQ(addresses.size() + old_addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(addresses.size() + old_addresses.size(), CurrentMetrics::get(metrics.active_count)); + + std::set results; + while (results.size() != addresses.size() + old_addresses.size()) + { + auto next_addr = resolver->resolve(); + results.insert(*next_addr); + } +} + +TEST_F(ResolvePoolTest, CanGainEven) +{ + auto resolver = make_resolver(); + auto address = resolver->resolve(); + + std::map results; + for (size_t i = 0; i < 40000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + ASSERT_GT(getMin(results), 10000); + + addresses.insert("127.0.0.4"); + addresses.insert("127.0.0.5"); + + resolver->update(); + + /// return mostly new addresses + for (size_t i = 0; i < 3000; ++i) + { + auto next_addr = resolver->resolve(); + if (results.contains(*next_addr)) + { + results[*next_addr] += 1; + } + else + { + results[*next_addr] = 1; + } + } + + ASSERT_EQ(results.size(), 5); + + ASSERT_GT(getMin(results), 1000); +} + +TEST_F(ResolvePoolTest, CanFail) +{ + auto resolver = make_resolver(10000); + + auto failed_addr = resolver->resolve(); + failed_addr.setFail(); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.failed]); + ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]); + + for (size_t i = 0; i < 1000; ++i) + { + auto next_addr = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*next_addr)); + ASSERT_NE(*next_addr, *failed_addr); + } +} + +TEST_F(ResolvePoolTest, CanFailAndHeal) +{ + auto resolver = make_resolver(); + + auto failed_addr = resolver->resolve(); + failed_addr.setFail(); + + while (true) + { + auto next_addr = resolver->resolve(); + if (*failed_addr == *next_addr) + break; + } +} + + +TEST_F(ResolvePoolTest, CanExpire) +{ + auto resolver = make_resolver(); + + auto expired_addr = resolver->resolve(); + ASSERT_TRUE(addresses.contains(*expired_addr)); + + addresses.erase(*expired_addr); + sleepForSeconds(1); + + for (size_t i = 0; i < 1000; ++i) + { + auto next_addr = resolver->resolve(); + + ASSERT_TRUE(addresses.contains(*next_addr)); + ASSERT_NE(*next_addr, *expired_addr); + } + + ASSERT_EQ(addresses.size() + 1, DB::CurrentThread::getProfileEvents()[metrics.discovered]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]); +} diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index fc478ae4f41..c201bab6063 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -128,8 +128,17 @@ namespace DB M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \ M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \ M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \ + M(UInt64, disk_connections_soft_limit, 1000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_store_limit, 12000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \ + M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \ + M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \ + M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \ + M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ + M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ + M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ - /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp +/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fceee63d4bb..d70a6cf51c5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -110,7 +110,7 @@ class IColumn; M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ - M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. This only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ + M(UInt64, s3_connect_timeout_ms, 1000, "Connection timeout for host from s3 disks.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 2f1da7935e6..e680c02671a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"24.3", {{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, + {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, + {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index bf42b7931ed..dae8ec06d30 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -88,20 +88,18 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) QueryPipeline HTTPDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); - Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + Poco::URI uri(configuration.url); + + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadUpdatedAll() @@ -109,19 +107,16 @@ QueryPipeline HTTPDictionarySource::loadUpdatedAll() Poco::URI uri(configuration.url); getUpdateFieldAndDate(uri); LOG_TRACE(log, "loadUpdatedAll {}", uri.toString()); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) @@ -139,19 +134,18 @@ QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) }; Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - out_stream_callback, - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withOutCallback(std::move(out_stream_callback)) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) @@ -169,19 +163,18 @@ QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const }; Poco::URI uri(configuration.url); - auto in_ptr = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - out_stream_callback, - timeouts, - credentials, - 0, - DBMS_DEFAULT_BUFFER_SIZE, - context->getReadSettings(), - configuration.header_entries, - nullptr, false); - return createWrappedBuffer(std::move(in_ptr)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(context->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(configuration.header_entries) + .withOutCallback(std::move(out_stream_callback)) + .withDelayInit(false) + .create(credentials); + + return createWrappedBuffer(std::move(buf)); } bool HTTPDictionarySource::isModified() const diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 70fe889a8ea..1ebfc4a29b0 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -203,7 +203,7 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) } -QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const +QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const { bridge_helper->startBridgeSync(); @@ -214,10 +214,15 @@ QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const B os << "query=" << escapeForFileName(query); }; - auto read_buf = std::make_unique( - url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts, credentials); - auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, max_block_size); - format->addBuffer(std::move(read_buf)); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(timeouts) + .withOutCallback(std::move(write_body_callback)) + .create(credentials); + + auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *buf, required_sample_block, max_block_size); + format->addBuffer(std::move(buf)); return QueryPipeline(std::move(format)); } diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 6011563c522..64d22807254 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -74,7 +74,7 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - QueryPipeline loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; + QueryPipeline loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const; LoggerPtr log; diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index cc872392738..7509aa81d75 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -1,8 +1,6 @@ #include "ReadBufferFromWebServer.h" #include -#include -#include #include #include #include @@ -45,12 +43,6 @@ std::unique_ptr ReadBufferFromWebServer::initialize() { if (read_until_position < offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - - LOG_DEBUG(log, "Reading with range: {}-{}", offset, read_until_position); - } - else - { - LOG_DEBUG(log, "Reading from offset: {}", offset); } const auto & settings = context->getSettingsRef(); @@ -60,19 +52,14 @@ std::unique_ptr ReadBufferFromWebServer::initialize() connection_timeouts.withConnectionTimeout(std::max(settings.http_connection_timeout, Poco::Timespan(20, 0))); connection_timeouts.withReceiveTimeout(std::max(settings.http_receive_timeout, Poco::Timespan(20, 0))); - auto res = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - ReadWriteBufferFromHTTP::OutStreamCallback(), - connection_timeouts, - credentials, - 0, - buf_size, - read_settings, - HTTPHeaderEntries{}, - &context->getRemoteHostFilter(), - /* delay_initialization */true, - use_external_buffer); + auto res = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::DISK) + .withSettings(read_settings) + .withTimeouts(connection_timeouts) + .withBufSize(buf_size) + .withHostFilter(&context->getRemoteHostFilter()) + .withExternalBuf(use_external_buffer) + .create(credentials); if (read_until_position) res->setReadUntilPosition(read_until_position); @@ -101,44 +88,44 @@ bool ReadBufferFromWebServer::nextImpl() throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); } - if (impl) - { - if (!use_external_buffer) - { - /** - * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read, because - * this branch means we read sequentially. - */ - impl->position() = position(); - assert(!impl->hasPendingData()); - } - } - else + if (!impl) { impl = initialize(); + + if (!use_external_buffer) + { + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + } } if (use_external_buffer) { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not, because this branch - * means we are prefetching data, each nextImpl() call we can fill - * a different buffer. - */ impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); + } + else + { + impl->position() = position(); } + chassert(available() == 0); + + chassert(pos >= working_buffer.begin()); + chassert(pos <= working_buffer.end()); + + chassert(working_buffer.begin() != nullptr); + chassert(impl->buffer().begin() != nullptr); + chassert(working_buffer.begin() == impl->buffer().begin()); + + chassert(impl->available() == 0); + auto result = impl->next(); + + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + + chassert(working_buffer.begin() == impl->buffer().begin()); + if (result) - { - BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); offset += working_buffer.size(); - } return result; } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4fd4b17aabe..7cc29bf1da2 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -67,10 +67,6 @@ std::unique_ptr getClient( client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms = config.getUInt( - config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); - client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); - client_configuration.wait_on_pool_size_limit = false; client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 48de0bf4168..0bad668a404 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -44,34 +44,33 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lockgetSettingsRef(), - getContext()->getServerSettings().keep_alive_timeout), - credentials, - /* max_redirects= */ 0, - /* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE, - getContext()->getReadSettings()); + auto timeouts = ConnectionTimeouts::getHTTPTimeouts( + getContext()->getSettingsRef(), + getContext()->getServerSettings().keep_alive_timeout); + + auto metadata_buf = BuilderRWBufferFromHTTP(Poco::URI(fs::path(full_url) / ".index")) + .withConnectionGroup(HTTPConnectionGroupType::DISK) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .create(credentials); String file_name; - while (!metadata_buf.eof()) + while (!metadata_buf->eof()) { - readText(file_name, metadata_buf); - assertChar('\t', metadata_buf); + readText(file_name, *metadata_buf); + assertChar('\t', *metadata_buf); bool is_directory; - readBoolText(is_directory, metadata_buf); + readBoolText(is_directory, *metadata_buf); size_t size = 0; if (!is_directory) { - assertChar('\t', metadata_buf); - readIntText(size, metadata_buf); + assertChar('\t', *metadata_buf); + readIntText(size, *metadata_buf); } - assertChar('\n', metadata_buf); + assertChar('\n', *metadata_buf); FileDataPtr file_data = is_directory ? FileData::createDirectoryInfo(false) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index a052f20d6fa..6679646fef4 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -122,7 +122,7 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { std::string_view sqid = col_non_const->getDataAt(i).toView(); - std::vector integers = sqids.decode(sqid); + std::vector integers = sqids.decode(String(sqid)); res_nested_data.insert(integers.begin(), integers.end()); res_offsets_data.push_back(integers.size()); } diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index f2db3169400..c4b636103fe 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -141,4 +141,19 @@ ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method .withReceiveTimeout(saturate(recv, receive_timeout)); } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) +{ + session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); +} + +ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session) +{ + return ConnectionTimeouts() + .withConnectionTimeout(session.getConnectionTimeout()) + .withSendTimeout(session.getSendTimeout()) + .withReceiveTimeout(session.getReceiveTimeout()) + .withHTTPKeepAliveTimeout(session.getKeepAliveTimeout()); +} + } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 7fe97b5ec36..2b2ab0e7ab8 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -4,6 +4,7 @@ #include #include +#include #include namespace DB @@ -111,4 +112,7 @@ inline ConnectionTimeouts & ConnectionTimeouts::withConnectionTimeout(Poco::Time return *this; } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); +ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session); + } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index c4468a1b896..09f7724d613 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -2,13 +2,7 @@ #include #include -#include -#include #include -#include -#include -#include -#include #include "config.h" @@ -25,338 +19,18 @@ #include -#include -#include +#include #include #include -namespace ProfileEvents -{ - extern const Event CreatedHTTPConnections; -} - namespace DB { + namespace ErrorCodes { extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER; extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; - extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME; - extern const int UNSUPPORTED_URI_SCHEME; - extern const int LOGICAL_ERROR; -} - - -namespace -{ - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration) - { - Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; - - poco_proxy_config.host = proxy_configuration.host; - poco_proxy_config.port = proxy_configuration.port; - poco_proxy_config.protocol = ProxyConfiguration::protocolToString(proxy_configuration.protocol); - poco_proxy_config.tunnel = proxy_configuration.tunneling; - poco_proxy_config.originalRequestProtocol = ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); - - return poco_proxy_config; - } - - template - requires std::derived_from - class HTTPSessionAdapter : public Session - { - static_assert(std::has_virtual_destructor_v, "The base class must have a virtual destructor"); - - public: - HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{getLogger("HTTPSessionAdapter")} { } - ~HTTPSessionAdapter() override = default; - - protected: - void reconnect() override - { - // First of all will try to establish connection with last used addr. - if (!Session::getResolvedHost().empty()) - { - try - { - Session::reconnect(); - return; - } - catch (...) - { - Session::close(); - LOG_TRACE( - log, - "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", - Session::getResolvedHost(), - Session::getHost(), - Session::getPort()); - } - } - - const auto endpoinds = DNSResolver::instance().resolveHostAll(Session::getHost()); - - for (auto it = endpoinds.begin();;) - { - try - { - Session::setResolvedHost(it->toString()); - Session::reconnect(); - - LOG_TRACE( - log, - "Created HTTP(S) session with {}:{} ({}:{})", - Session::getHost(), - Session::getPort(), - it->toString(), - Session::getPort()); - - break; - } - catch (...) - { - Session::close(); - if (++it == endpoinds.end()) - { - Session::setResolvedHost(""); - throw; - } - LOG_TRACE( - log, - "Failed to create connection with {}:{}, Will try another resolved address. {}", - Session::getResolvedHost(), - Session::getPort(), - getCurrentExceptionMessage(false)); - } - } - } - LoggerPtr log; - }; - - bool isHTTPS(const Poco::URI & uri) - { - if (uri.getScheme() == "https") - return true; - else if (uri.getScheme() == "http") - return false; - else - throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); - } - - HTTPSessionPtr makeHTTPSessionImpl( - const std::string & host, - UInt16 port, - bool https, - bool keep_alive, - DB::ProxyConfiguration proxy_configuration = {}) - { - HTTPSessionPtr session; - - if (!proxy_configuration.host.empty()) - { - bool is_proxy_http_and_is_tunneling_off = DB::ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol - && !proxy_configuration.tunneling; - - // If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request. - // The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server. - // There is a weak link in the security, but that's what the user opted for. - if (https && is_proxy_http_and_is_tunneling_off) - { - https = false; - } - } - - if (https) - { -#if USE_SSL - session = std::make_shared>(host, port); -#else - throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without HTTPS support"); -#endif - } - else - { - session = std::make_shared>(host, port); - } - - ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - - /// doesn't work properly without patch - session->setKeepAlive(keep_alive); - - if (!proxy_configuration.host.empty()) - { - session->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); - } - - return session; - } - - class SingleEndpointHTTPSessionPool : public PoolBase - { - private: - const std::string host; - const UInt16 port; - const bool https; - ProxyConfiguration proxy_config; - - using Base = PoolBase; - - ObjectPtr allocObject() override - { - /// Pool is global, we shouldn't attribute this memory to query/user. - MemoryTrackerSwitcher switcher{&total_memory_tracker}; - - auto session = makeHTTPSessionImpl(host, port, https, true, proxy_config); - return session; - } - - public: - SingleEndpointHTTPSessionPool( - const std::string & host_, - UInt16 port_, - bool https_, - ProxyConfiguration proxy_config_, - size_t max_pool_size_, - bool wait_on_pool_size_limit) - : Base( - static_cast(max_pool_size_), - getLogger("HTTPSessionPool"), - wait_on_pool_size_limit ? BehaviourOnLimit::Wait : BehaviourOnLimit::AllocateNewBypassingPool) - , host(host_) - , port(port_) - , https(https_) - , proxy_config(proxy_config_) - { - } - }; - - class HTTPSessionPool : private boost::noncopyable - { - public: - struct Key - { - String target_host; - UInt16 target_port; - bool is_target_https; - ProxyConfiguration proxy_config; - bool wait_on_pool_size_limit; - - bool operator ==(const Key & rhs) const - { - return std::tie( - target_host, - target_port, - is_target_https, - proxy_config.host, - proxy_config.port, - proxy_config.protocol, - proxy_config.tunneling, - proxy_config.original_request_protocol, - wait_on_pool_size_limit) - == std::tie( - rhs.target_host, - rhs.target_port, - rhs.is_target_https, - rhs.proxy_config.host, - rhs.proxy_config.port, - rhs.proxy_config.protocol, - rhs.proxy_config.tunneling, - rhs.proxy_config.original_request_protocol, - rhs.wait_on_pool_size_limit); - } - }; - - private: - using PoolPtr = std::shared_ptr; - using Entry = SingleEndpointHTTPSessionPool::Entry; - - struct Hasher - { - size_t operator()(const Key & k) const - { - SipHash s; - s.update(k.target_host); - s.update(k.target_port); - s.update(k.is_target_https); - s.update(k.proxy_config.host); - s.update(k.proxy_config.port); - s.update(k.proxy_config.protocol); - s.update(k.proxy_config.tunneling); - s.update(k.proxy_config.original_request_protocol); - s.update(k.wait_on_pool_size_limit); - return s.get64(); - } - }; - - std::mutex mutex; - std::unordered_map endpoints_pool; - - protected: - HTTPSessionPool() = default; - - public: - static auto & instance() - { - static HTTPSessionPool instance; - return instance; - } - - Entry getSession( - const Poco::URI & uri, - const ProxyConfiguration & proxy_config, - const ConnectionTimeouts & timeouts, - size_t max_connections_per_endpoint, - bool wait_on_pool_size_limit) - { - std::unique_lock lock(mutex); - const std::string & host = uri.getHost(); - UInt16 port = uri.getPort(); - bool https = isHTTPS(uri); - - HTTPSessionPool::Key key{host, port, https, proxy_config, wait_on_pool_size_limit}; - auto pool_ptr = endpoints_pool.find(key); - if (pool_ptr == endpoints_pool.end()) - std::tie(pool_ptr, std::ignore) = endpoints_pool.emplace( - key, - std::make_shared( - host, - port, - https, - proxy_config, - max_connections_per_endpoint, - wait_on_pool_size_limit)); - - /// Some routines held session objects until the end of its lifetime. Also this routines may create another sessions in this time frame. - /// If some other session holds `lock` because it waits on another lock inside `pool_ptr->second->get` it isn't possible to create any - /// new session and thus finish routine, return session to the pool and unlock the thread waiting inside `pool_ptr->second->get`. - /// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`. - lock.unlock(); - - auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); - auto session = pool_ptr->second->get(retry_timeout); - - const auto & session_data = session->sessionData(); - if (session_data.empty() || !Poco::AnyCast(&session_data)) - { - /// Reset session if it is not reusable. See comment for HTTPSessionReuseTag. - session->reset(); - } - session->attachSessionData({}); - - setTimeouts(*session, timeouts); - - return session; - } - }; -} - -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) -{ - session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); } void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) @@ -370,28 +44,13 @@ void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_ } HTTPSessionPtr makeHTTPSession( + HTTPConnectionGroupType group, const Poco::URI & uri, const ConnectionTimeouts & timeouts, - ProxyConfiguration proxy_configuration -) + ProxyConfiguration proxy_configuration) { - const std::string & host = uri.getHost(); - UInt16 port = uri.getPort(); - bool https = isHTTPS(uri); - - auto session = makeHTTPSessionImpl(host, port, https, false, proxy_configuration); - setTimeouts(*session, timeouts); - return session; -} - -PooledHTTPSessionPtr makePooledHTTPSession( - const Poco::URI & uri, - const ConnectionTimeouts & timeouts, - size_t per_endpoint_pool_size, - bool wait_on_pool_size_limit, - ProxyConfiguration proxy_config) -{ - return HTTPSessionPool::instance().getSession(uri, proxy_config, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); + auto connection_pool = HTTPConnectionPools::instance().getPool(group, uri, proxy_configuration); + return connection_pool->getConnection(timeouts); } bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; } @@ -400,11 +59,11 @@ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, const bool allow_redirects) { auto & istr = session.receiveResponse(response); - assertResponseIsOk(request, response, istr, allow_redirects); + assertResponseIsOk(request.getURI(), response, istr, allow_redirects); return &istr; } -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects) +void assertResponseIsOk(const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects) { auto status = response.getStatus(); @@ -422,7 +81,7 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR body.exceptions(std::ios::failbit); body << istr.rdbuf(); - throw HTTPException(code, request.getURI(), status, response.getReason(), body.str()); + throw HTTPException(code, uri, status, response.getReason(), body.str()); } } @@ -440,24 +99,4 @@ Exception HTTPException::makeExceptionMessage( uri, static_cast(http_status), reason, body); } -void markSessionForReuse(Poco::Net::HTTPSession & session) -{ - const auto & session_data = session.sessionData(); - if (!session_data.empty() && !Poco::AnyCast(&session_data)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Data of an unexpected type ({}) is attached to the session", session_data.type().name()); - - session.attachSessionData(HTTPSessionReuseTag{}); -} - -void markSessionForReuse(HTTPSessionPtr session) -{ - markSessionForReuse(*session); -} - -void markSessionForReuse(PooledHTTPSessionPtr session) -{ - markSessionForReuse(static_cast(*session)); -} - } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index c9968fc6915..e27269e2559 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -7,9 +7,9 @@ #include #include #include -#include -#include #include +#include +#include #include @@ -36,7 +36,7 @@ public: HTTPException * clone() const override { return new HTTPException(*this); } void rethrow() const override { throw *this; } - int getHTTPStatus() const { return http_status; } + Poco::Net::HTTPResponse::HTTPStatus getHTTPStatus() const { return http_status; } private: Poco::Net::HTTPResponse::HTTPStatus http_status{}; @@ -52,55 +52,18 @@ private: const char * className() const noexcept override { return "DB::HTTPException"; } }; -using PooledHTTPSessionPtr = PoolBase::Entry; // SingleEndpointHTTPSessionPool::Entry using HTTPSessionPtr = std::shared_ptr; -/// If a session have this tag attached, it will be reused without calling `reset()` on it. -/// All pooled sessions don't have this tag attached after being taken from a pool. -/// If the request and the response were fully written/read, the client code should add this tag -/// explicitly by calling `markSessionForReuse()`. -/// -/// Note that HTTP response may contain extra bytes after the last byte of the payload. Specifically, -/// when chunked encoding is used, there's an empty chunk at the end. Those extra bytes must also be -/// read before the session can be reused. So we usually put an `istr->ignore(INT64_MAX)` call -/// before `markSessionForReuse()`. -struct HTTPSessionReuseTag -{ -}; - -void markSessionForReuse(Poco::Net::HTTPSession & session); -void markSessionForReuse(HTTPSessionPtr session); -void markSessionForReuse(PooledHTTPSessionPtr session); - - void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession( + HTTPConnectionGroupType group, const Poco::URI & uri, const ConnectionTimeouts & timeouts, ProxyConfiguration proxy_config = {} ); -/// As previous method creates session, but takes it from pool, without and with proxy uri. -/// -/// The max_connections_per_endpoint parameter makes it look like the pool size can be different for -/// different requests (whatever that means), but actually we just assign the endpoint's connection -/// pool size when we see the endpoint for the first time, then we never change it. -/// We should probably change how this configuration works, and how this pooling works in general: -/// * Make the per_endpoint_pool_size be a global server setting instead of per-disk or per-query. -/// * Have boolean per-disk/per-query settings for enabling/disabling pooling. -/// * Add a limit on the number of endpoints and the total number of sessions across all endpoints. -/// * Enable pooling by default everywhere. In particular StorageURL and StorageS3. -/// (Enabling it for StorageURL is scary without the previous item - the user may query lots of -/// different endpoints. So currently pooling is mainly used for S3.) -PooledHTTPSessionPtr makePooledHTTPSession( - const Poco::URI & uri, - const ConnectionTimeouts & timeouts, - size_t per_endpoint_pool_size, - bool wait_on_pool_size_limit = true, - ProxyConfiguration proxy_config = {}); - bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status); /** Used to receive response (response headers and possibly body) @@ -112,7 +75,6 @@ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects); void assertResponseIsOk( - const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); + const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); } diff --git a/src/IO/LimitReadBuffer.cpp b/src/IO/LimitReadBuffer.cpp index e14112f8d19..84c7ac86227 100644 --- a/src/IO/LimitReadBuffer.cpp +++ b/src/IO/LimitReadBuffer.cpp @@ -1,5 +1,4 @@ #include - #include @@ -15,7 +14,7 @@ namespace ErrorCodes bool LimitReadBuffer::nextImpl() { - assert(position() >= in->position()); + chassert(position() >= in->position()); /// Let underlying buffer calculate read bytes in `next()` call. in->position() = position(); @@ -39,20 +38,18 @@ bool LimitReadBuffer::nextImpl() if (exact_limit && bytes != *exact_limit) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, *exact_limit); /// Clearing the buffer with existing data. - set(in->position(), 0); + BufferBase::set(in->position(), 0, 0); + return false; } - working_buffer = in->buffer(); - - if (limit - bytes < working_buffer.size()) - working_buffer.resize(limit - bytes); + BufferBase::set(in->position(), std::min(in->available(), limit - bytes), 0); return true; } -LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : ReadBuffer(in_ ? in_->position() : nullptr, 0) , in(in_) @@ -62,24 +59,20 @@ LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, boo , exact_limit(exact_limit_) , exception_message(std::move(exception_message_)) { - assert(in); + chassert(in); - size_t remaining_bytes_in_buffer = in->buffer().end() - in->position(); - if (remaining_bytes_in_buffer > limit) - remaining_bytes_in_buffer = limit; - - working_buffer = Buffer(in->position(), in->position() + remaining_bytes_in_buffer); + BufferBase::set(in->position(), std::min(in->available(), limit), 0); } -LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_) { } -LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, +LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_) { diff --git a/src/IO/LimitReadBuffer.h b/src/IO/LimitReadBuffer.h index 15885c1d850..b869f2935fb 100644 --- a/src/IO/LimitReadBuffer.h +++ b/src/IO/LimitReadBuffer.h @@ -13,22 +13,24 @@ namespace DB class LimitReadBuffer : public ReadBuffer { public: - LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, + LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_ = {}); - LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, + LimitReadBuffer(std::unique_ptr in_, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_ = {}); ~LimitReadBuffer() override; private: ReadBuffer * in; - bool owns_in; + const bool owns_in; - UInt64 limit; - bool throw_exception; - std::optional exact_limit; - std::string exception_message; + const size_t limit; + const bool throw_exception; + const std::optional exact_limit; + const std::string exception_message; - LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_); + LoggerPtr log; + + LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_); bool nextImpl() override; }; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 9b1c132cc01..f27828f71b2 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -92,7 +92,7 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(getFD(), getFileName()); } -size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) const { if (offset >= mapped.getLength()) return 0; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 2a039e04971..f774538374a 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -40,7 +40,7 @@ public: size_t getFileSize() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) const override; bool supportsReadAt() override { return true; } }; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 00325734354..056e25a5fbe 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -63,21 +63,23 @@ public: */ bool next() { - assert(!hasPendingData()); - assert(position() <= working_buffer.end()); + chassert(!hasPendingData()); + chassert(position() <= working_buffer.end()); bytes += offset(); bool res = nextImpl(); if (!res) + { working_buffer = Buffer(pos, pos); + } else { - pos = working_buffer.begin() + nextimpl_working_buffer_offset; - assert(position() != working_buffer.end()); + pos = working_buffer.begin() + std::min(nextimpl_working_buffer_offset, working_buffer.size()); + chassert(position() < working_buffer.end()); } nextimpl_working_buffer_offset = 0; - assert(position() <= working_buffer.end()); + chassert(position() <= working_buffer.end()); return res; } diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 3211f8eeb35..57442a15853 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -49,7 +49,7 @@ std::string ReadBufferFromFileDescriptor::getFileName() const } -size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) +size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const { chassert(min_bytes <= max_bytes); @@ -265,7 +265,7 @@ bool ReadBufferFromFileDescriptor::checkIfActuallySeekable() return res == 0 && S_ISREG(stat.st_mode); } -size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) const { chassert(use_pread); return readImpl(to, n, n, offset); diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 4762998c67b..db256ef91c7 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -34,7 +34,7 @@ protected: /// Doesn't seek (`offset` must match fd's position if !use_pread). /// Stops after min_bytes or eof. Returns 0 if eof. /// Thread safe. - size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset); + size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const; public: explicit ReadBufferFromFileDescriptor( @@ -73,7 +73,7 @@ public: bool checkIfActuallySeekable() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) const override; bool supportsReadAt() override { return use_pread; } }; diff --git a/src/IO/ReadBufferFromIStream.cpp b/src/IO/ReadBufferFromIStream.cpp index 52546f1703d..bc90ec7ed15 100644 --- a/src/IO/ReadBufferFromIStream.cpp +++ b/src/IO/ReadBufferFromIStream.cpp @@ -5,52 +5,44 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_READ_FROM_ISTREAM; -} - bool ReadBufferFromIStream::nextImpl() { if (eof) return false; + chassert(internal_buffer.begin() != nullptr); + chassert(!internal_buffer.empty()); + size_t bytes_read = 0; char * read_to = internal_buffer.begin(); /// It is necessary to read in a loop, since socket usually returns only data available at the moment. while (bytes_read < internal_buffer.size()) { - try + const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read); + if (bytes_read_last_time <= 0) { - const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read); - if (bytes_read_last_time <= 0) - { - eof = true; - break; - } + eof = true; + break; + } - bytes_read += bytes_read_last_time; - read_to += bytes_read_last_time; - } - catch (...) - { - throw Exception( - ErrorCodes::CANNOT_READ_FROM_ISTREAM, - "Cannot read from istream at offset {}: {}", - count(), - getCurrentExceptionMessage(/*with_stacktrace=*/true)); - } + bytes_read += bytes_read_last_time; + read_to += bytes_read_last_time; } if (bytes_read) + { + working_buffer = internal_buffer; working_buffer.resize(bytes_read); + } return bytes_read; } ReadBufferFromIStream::ReadBufferFromIStream(std::istream & istr_, size_t size) - : BufferWithOwnMemory(size), istr(istr_), stream_buf(dynamic_cast(*istr.rdbuf())) + : BufferWithOwnMemory(size) + , istr(istr_) + , stream_buf(dynamic_cast(*istr.rdbuf())) { } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 4529771e7b2..491ff253066 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -34,61 +34,6 @@ namespace ProfileEvents extern const Event RemoteReadThrottlerSleepMicroseconds; } -namespace -{ -DB::PooledHTTPSessionPtr getSession(Aws::S3::Model::GetObjectResult & read_result) -{ - if (auto * session_aware_stream = dynamic_cast *>(&read_result.GetBody())) - return static_cast(session_aware_stream->getSession()); - - if (dynamic_cast *>(&read_result.GetBody())) - return {}; - - /// accept result from S# mock in gtest_writebuffer_s3.cpp - if (dynamic_cast(&read_result.GetBody())) - return {}; - - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session of unexpected type encountered"); -} - -void resetSession(Aws::S3::Model::GetObjectResult & read_result) -{ - if (auto session = getSession(read_result); !session.isNull()) - { - auto & http_session = static_cast(*session); - http_session.reset(); - } -} - -void resetSessionIfNeeded(bool read_all_range_successfully, std::optional & read_result) -{ - if (!read_result) - return; - - if (!read_all_range_successfully) - { - /// When we abandon a session with an ongoing GetObject request and there is another one trying to delete the same object this delete - /// operation will hang until GetObject's session idle timeouts. So we have to call `reset()` on GetObject's session session immediately. - resetSession(*read_result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - else if (auto session = getSession(*read_result); !session.isNull()) - { - if (!session->getProxyHost().empty()) - { - /// Reset proxified sessions because proxy can change for every request. See ProxyConfigurationResolver. - resetSession(*read_result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - else - { - DB::markSessionForReuse(session); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); - } - } -} -} - namespace DB { namespace ErrorCodes @@ -228,7 +173,7 @@ bool ReadBufferFromS3::nextImpl() } -size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const { size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; @@ -240,29 +185,6 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); std::optional result; - /// Connection is reusable if we've read the full response. - bool session_is_reusable = false; - SCOPE_EXIT( - { - if (!result.has_value()) - return; - if (session_is_reusable) - { - auto session = getSession(*result); - if (!session.isNull()) - { - DB::markSessionForReuse(session); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); - } - else - session_is_reusable = false; - } - if (!session_is_reusable) - { - resetSession(*result); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); - } - }); try { @@ -276,9 +198,8 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - /// Read remaining bytes after the end of the payload, see HTTPSessionReuseTag. + /// Read remaining bytes after the end of the payload istr.ignore(INT64_MAX); - session_is_reusable = true; } catch (Poco::Exception & e) { @@ -451,21 +372,8 @@ bool ReadBufferFromS3::atEndOfRequestedRangeGuess() return false; } -ReadBufferFromS3::~ReadBufferFromS3() -{ - try - { - resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); - } - catch (...) - { - tryLogCurrentException(log); - } -} - std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) { - resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); read_all_range_successfully = false; /** @@ -534,10 +442,6 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si } } -bool ReadBufferFromS3::readAllRangeSuccessfully() const -{ - return read_until_position ? offset == read_until_position : read_all_range_successfully; -} } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index f28c23a71d7..003c88df7d2 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -55,7 +55,7 @@ public: bool restricted_seek_ = false, std::optional file_size = std::nullopt); - ~ReadBufferFromS3() override; + ~ReadBufferFromS3() override = default; bool nextImpl() override; @@ -74,7 +74,7 @@ public: String getFileName() const override { return bucket + "/" + key; } - size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const override; bool supportsReadAt() override { return true; } @@ -90,8 +90,6 @@ private: Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const; - bool readAllRangeSuccessfully() const; - ReadSettings read_settings; bool use_external_buffer; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 31ea45d92a9..38904df4403 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -122,7 +122,7 @@ struct ReadSettings // Resource to be used during reading ResourceLink resource_link; - size_t http_max_tries = 1; + size_t http_max_tries = 10; size_t http_retry_initial_backoff_ms = 100; size_t http_retry_max_backoff_ms = 1600; bool http_skip_not_found_url_for_globs = true; diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 15c2a0a021b..bcbec97537a 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -1,13 +1,65 @@ #include "ReadWriteBufferFromHTTP.h" #include +#include +#include + namespace ProfileEvents { extern const Event ReadBufferSeekCancelConnection; - extern const Event ReadWriteBufferFromHTTPPreservedSessions; } + +namespace +{ + +bool isRetriableError(const Poco::Net::HTTPResponse::HTTPStatus http_status) noexcept +{ + static constexpr std::array non_retriable_errors{ + Poco::Net::HTTPResponse::HTTPStatus::HTTP_BAD_REQUEST, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_IMPLEMENTED, + Poco::Net::HTTPResponse::HTTPStatus::HTTP_METHOD_NOT_ALLOWED}; + + return std::all_of( + non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); +} + +Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) +{ + chassert(DB::isRedirect(response.getStatus())); + + auto location = response.get("Location"); + auto location_uri = Poco::URI(location); + if (!location_uri.isRelative()) + return location_uri; + /// Location header contains relative path. So we need to concatenate it + /// with path from the original URI and normalize it. + auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location); + location_uri = prev_uri; + location_uri.setPath(path); + return location_uri; +} + +class ReadBufferFromSessionResponse : public DB::ReadBufferFromIStream +{ +private: + DB::HTTPSessionPtr session; + +public: + ReadBufferFromSessionResponse(DB::HTTPSessionPtr && session_, std::istream & rstr, size_t size) + : ReadBufferFromIStream(rstr, size) + , session(std::move(session_)) + { + } +}; + +} + + namespace DB { @@ -21,94 +73,29 @@ namespace ErrorCodes extern const int UNKNOWN_FILE_SIZE; } -template -UpdatableSession::UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr session_factory_) - : max_redirects{max_redirects_} - , initial_uri(uri) - , session_factory(std::move(session_factory_)) +std::unique_ptr ReadWriteBufferFromHTTP::CallResult::transformToReadBuffer(size_t buf_size) && { - session = session_factory->buildNewSession(uri); + chassert(session); + return std::make_unique(std::move(session), *response_stream, buf_size); } -template -typename UpdatableSession::SessionPtr UpdatableSession::getSession() { return session; } - -template -void UpdatableSession::updateSession(const Poco::URI & uri) -{ - ++redirects; - if (redirects <= max_redirects) - session = session_factory->buildNewSession(uri); - else - throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, - "Too many redirects while trying to access {}." - " You can {} redirects by changing the setting 'max_http_get_redirects'." - " Example: `SET max_http_get_redirects = 10`." - " Redirects are restricted to prevent possible attack when a malicious server redirects to an internal resource, bypassing the authentication or firewall.", - initial_uri.toString(), max_redirects ? "increase the allowed maximum number of" : "allow"); -} - -template -typename UpdatableSession::SessionPtr UpdatableSession::createDetachedSession(const Poco::URI & uri) -{ - return session_factory->buildNewSession(uri); -} - -template -std::shared_ptr> UpdatableSession::clone(const Poco::URI & uri) -{ - return std::make_shared>(uri, max_redirects, session_factory); -} - - -namespace detail -{ - -static bool isRetriableError(const Poco::Net::HTTPResponse::HTTPStatus http_status) noexcept -{ - static constexpr std::array non_retriable_errors{ - Poco::Net::HTTPResponse::HTTPStatus::HTTP_BAD_REQUEST, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_IMPLEMENTED, - Poco::Net::HTTPResponse::HTTPStatus::HTTP_METHOD_NOT_ALLOWED}; - - return std::all_of( - non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); -} - -static Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) -{ - auto location = response.get("Location"); - auto location_uri = Poco::URI(location); - if (!location_uri.isRelative()) - return location_uri; - /// Location header contains relative path. So we need to concatenate it - /// with path from the original URI and normalize it. - auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location); - location_uri = prev_uri; - location_uri.setPath(path); - return location_uri; -} - -template -bool ReadWriteBufferFromHTTPBase::withPartialContent(const HTTPRange & range) const +bool ReadWriteBufferFromHTTP::withPartialContent() const { /** * Add range header if we have some passed range * or if we want to retry GET request on purpose. */ - return range.begin || range.end || retry_with_range_header; + return read_range.begin || read_range.end || getOffset() > 0; } -template -size_t ReadWriteBufferFromHTTPBase::getOffset() const { return read_range.begin.value_or(0) + offset_from_begin_pos; } - -template -void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const +size_t ReadWriteBufferFromHTTP::getOffset() const { - request.setHost(uri_.getHost()); // use original, not resolved host name in header + return read_range.begin.value_or(0) + offset_from_begin_pos; +} + +void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, std::optional range) const +{ + request.setHost(initial_uri.getHost()); // use original, not resolved host name in header if (out_stream_callback) request.setChunkedTransferEncoding(true); @@ -125,7 +112,6 @@ void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net: range_header_value = fmt::format("bytes={}-{}", *range->begin, *range->end); else range_header_value = fmt::format("bytes={}-", *range->begin); - LOG_TEST(log, "Adding header: Range: {}", range_header_value); request.set("Range", range_header_value); } @@ -133,45 +119,7 @@ void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net: credentials.authenticate(request); } -template -std::istream * ReadWriteBufferFromHTTPBase::callImpl( - UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info) -{ - // With empty path poco will send "POST HTTP/1.1" its bug. - if (uri_.getPath().empty()) - uri_.setPath("/"); - - std::optional range; - if (!for_object_info) - { - if (withPartialContent(read_range)) - range = HTTPRange{getOffset(), read_range.end}; - } - - Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - prepareRequest(request, uri_, range); - - LOG_TRACE(log, "Sending request to {}", uri_.toString()); - - auto sess = current_session->getSession(); - auto & stream_out = sess->sendRequest(request); - - if (out_stream_callback) - out_stream_callback(stream_out); - - auto result_istr = receiveResponse(*sess, request, response, true); - response.getCookies(cookies); - - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (!for_object_info) - content_encoding = response.get("Content-Encoding", ""); - - return result_istr; -} - -template -size_t ReadWriteBufferFromHTTPBase::getFileSize() +size_t ReadWriteBufferFromHTTP::getFileSize() { if (!file_info) file_info = getFileInfo(); @@ -179,243 +127,288 @@ size_t ReadWriteBufferFromHTTPBase::getFileSize() if (file_info->file_size) return *file_info->file_size; - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", initial_uri.toString()); } -template -bool ReadWriteBufferFromHTTPBase::supportsReadAt() +bool ReadWriteBufferFromHTTP::supportsReadAt() { if (!file_info) file_info = getFileInfo(); return method == Poco::Net::HTTPRequest::HTTP_GET && file_info->seekable; } -template -bool ReadWriteBufferFromHTTPBase::checkIfActuallySeekable() +bool ReadWriteBufferFromHTTP::checkIfActuallySeekable() { if (!file_info) file_info = getFileInfo(); return file_info->seekable; } -template -String ReadWriteBufferFromHTTPBase::getFileName() const { return uri.toString(); } - -template -void ReadWriteBufferFromHTTPBase::getHeadResponse(Poco::Net::HTTPResponse & response) +String ReadWriteBufferFromHTTP::getFileName() const { - for (size_t i = 0; i < settings.http_max_tries; ++i) - { - try - { - callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD, true, true); - break; - } - catch (const Poco::Exception & e) - { - if (i == settings.http_max_tries - 1 || e.code() == ErrorCodes::TOO_MANY_REDIRECTS || !isRetriableError(response.getStatus())) - throw; - - LOG_ERROR(log, "Failed to make HTTP_HEAD request to {}. Error: {}", uri.toString(), e.displayText()); - } - } + return initial_uri.toString(); } -template -void ReadWriteBufferFromHTTPBase::setupExternalBuffer() +void ReadWriteBufferFromHTTP::getHeadResponse(Poco::Net::HTTPResponse & response) { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not (no hasPendingData() check is needed), - * because this branch means we are prefetching data, - * each nextImpl() call we can fill a different buffer. - */ - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); + doWithRetries( + [&] () + { + callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD, {}); + }, + /*on_retry=*/ nullptr, + /*mute_logging=*/ true); } -template -ReadWriteBufferFromHTTPBase::ReadWriteBufferFromHTTPBase( - UpdatableSessionPtr session_, - Poco::URI uri_, - const Poco::Net::HTTPBasicCredentials & credentials_, +ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( + const HTTPConnectionGroupType & connection_group_, + const Poco::URI & uri_, const std::string & method_, - OutStreamCallback out_stream_callback_, - size_t buffer_size_, - const ReadSettings & settings_, - HTTPHeaderEntries http_header_entries_, + ProxyConfiguration proxy_config_, + ReadSettings read_settings_, + ConnectionTimeouts timeouts_, + const Poco::Net::HTTPBasicCredentials & credentials_, const RemoteHostFilter * remote_host_filter_, - bool delay_initialization, + size_t buffer_size_, + size_t max_redirects_, + OutStreamCallback out_stream_callback_, bool use_external_buffer_, bool http_skip_not_found_url_, - std::optional file_info_, - ProxyConfiguration proxy_config_) + HTTPHeaderEntries http_header_entries_, + bool delay_initialization, + std::optional file_info_) : SeekableReadBuffer(nullptr, 0) - , uri {uri_} - , method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} - , session {session_} - , out_stream_callback {out_stream_callback_} - , credentials {credentials_} - , http_header_entries {std::move(http_header_entries_)} - , remote_host_filter {remote_host_filter_} - , buffer_size {buffer_size_} - , use_external_buffer {use_external_buffer_} - , file_info(file_info_) + , connection_group(connection_group_) + , initial_uri(uri_) + , method(!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET) + , proxy_config(std::move(proxy_config_)) + , read_settings(std::move(read_settings_)) + , timeouts(std::move(timeouts_)) + , credentials(credentials_) + , remote_host_filter(remote_host_filter_) + , buffer_size(buffer_size_) + , max_redirects(max_redirects_) + , use_external_buffer(use_external_buffer_) , http_skip_not_found_url(http_skip_not_found_url_) - , settings {settings_} + , out_stream_callback(std::move(out_stream_callback_)) + , redirects(0) + , http_header_entries {std::move(http_header_entries_)} + , file_info(file_info_) , log(getLogger("ReadWriteBufferFromHTTP")) - , proxy_config(proxy_config_) { - if (settings.http_max_tries <= 0 || settings.http_retry_initial_backoff_ms <= 0 - || settings.http_retry_initial_backoff_ms >= settings.http_retry_max_backoff_ms) + current_uri = initial_uri; + + if (current_uri.getPath().empty()) + current_uri.setPath("/"); + + if (read_settings.http_max_tries <= 0 || read_settings.http_retry_initial_backoff_ms <= 0 + || read_settings.http_retry_initial_backoff_ms >= read_settings.http_retry_max_backoff_ms) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Invalid setting for http backoff, " "must be http_max_tries >= 1 (current is {}) and " "0 < http_retry_initial_backoff_ms < settings.http_retry_max_backoff_ms (now 0 < {} < {})", - settings.http_max_tries, - settings.http_retry_initial_backoff_ms, - settings.http_retry_max_backoff_ms); + read_settings.http_max_tries, + read_settings.http_retry_initial_backoff_ms, + read_settings.http_retry_max_backoff_ms); // Configure User-Agent if it not already set. const std::string user_agent = "User-Agent"; - auto iter = std::find_if( - http_header_entries.begin(), - http_header_entries.end(), - [&user_agent](const HTTPHeaderEntry & entry) { return entry.name == user_agent; }); + auto iter = std::find_if(http_header_entries.begin(), http_header_entries.end(), + [&user_agent] (const HTTPHeaderEntry & entry) { return entry.name == user_agent; }); if (iter == http_header_entries.end()) { - http_header_entries.emplace_back("User-Agent", fmt::format("ClickHouse/{}", VERSION_STRING)); + http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING)); } + if (!delay_initialization && use_external_buffer) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid setting for ReadWriteBufferFromHTTP" + "delay_initialization is false and use_external_buffer it true."); + if (!delay_initialization) { - initialize(); - if (exception) - std::rethrow_exception(exception); + next(); } } -template -void ReadWriteBufferFromHTTPBase::callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors, bool for_object_info) +ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callImpl( + Poco::Net::HTTPResponse & response, const Poco::URI & uri_, const std::string & method_, const std::optional & range, bool allow_redirects) const { - UpdatableSessionPtr current_session = nullptr; + if (remote_host_filter) + remote_host_filter->checkURL(uri_); - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (for_object_info) - current_session = session->clone(uri); - else - current_session = session; + Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, range); - call(current_session, response, method_, throw_on_all_errors, for_object_info); - saved_uri_redirect = uri; + auto session = makeHTTPSession(connection_group, uri_, timeouts, proxy_config); + + auto & stream_out = session->sendRequest(request); + if (out_stream_callback) + out_stream_callback(stream_out); + + auto & resp_stream = session->receiveResponse(response); + + assertResponseIsOk(current_uri.toString(), response, resp_stream, allow_redirects); + + return ReadWriteBufferFromHTTP::CallResult(std::move(session), resp_stream); +} + +ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callWithRedirects( + Poco::Net::HTTPResponse & response, const String & method_, const std::optional & range) +{ + auto result = callImpl(response, current_uri, method_, range, true); while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect = getUriAfterRedirect(*saved_uri_redirect, response); - saved_uri_redirect = uri_redirect; - if (remote_host_filter) - remote_host_filter->checkURL(uri_redirect); + Poco::URI uri_redirect = getUriAfterRedirect(current_uri, response); + ++redirects; + if (redirects > max_redirects) + throw Exception( + ErrorCodes::TOO_MANY_REDIRECTS, + "Too many redirects while trying to access {}." + " You can {} redirects by changing the setting 'max_http_get_redirects'." + " Example: `SET max_http_get_redirects = 10`." + " Redirects are restricted to prevent possible attack when a malicious server redirects to an internal resource, bypassing the authentication or firewall.", + initial_uri.toString(), max_redirects ? "increase the allowed maximum number of" : "allow"); - current_session->updateSession(uri_redirect); + current_uri = uri_redirect; - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - auto result_istr = callImpl(current_session, uri_redirect, response, method, for_object_info); - if (!for_object_info) - istr = result_istr; + result = callImpl(response, uri_redirect, method_, range, true); } + + return result; } -template -void ReadWriteBufferFromHTTPBase::call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors, bool for_object_info) + +void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, + std::function on_retry, + bool mute_logging) const { - try + [[maybe_unused]] auto milliseconds_to_wait = read_settings.http_retry_initial_backoff_ms; + + bool is_retriable = true; + std::exception_ptr exception = nullptr; + + for (size_t attempt = 1; attempt <= read_settings.http_max_tries; ++attempt) { - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - auto result_istr = callImpl(current_session, saved_uri_redirect ? *saved_uri_redirect : uri, response, method_, for_object_info); - if (!for_object_info) - istr = result_istr; - } - catch (...) - { - /// we can fetch object info while the request is being processed - /// and we don't want to override any context used by it - if (for_object_info) - throw; + [[maybe_unused]] bool last_attempt = attempt + 1 > read_settings.http_max_tries; - if (throw_on_all_errors) - throw; + String error_message; - auto http_status = response.getStatus(); - - if (http_status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND && http_skip_not_found_url) + try { - initialization_error = InitializeError::SKIP_NOT_FOUND_URL; + callable(); + return; } - else if (!isRetriableError(http_status)) + catch (Poco::Net::NetException & e) { - initialization_error = InitializeError::NON_RETRYABLE_ERROR; + error_message = e.displayText(); exception = std::current_exception(); } + catch (DB::NetException & e) + { + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (DB::HTTPException & e) + { + if (!isRetriableError(e.getHTTPStatus())) + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (DB::Exception & e) + { + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + catch (Poco::Exception & e) + { + if (e.code() == POCO_EMFILE) + is_retriable = false; + + error_message = e.displayText(); + exception = std::current_exception(); + } + + chassert(exception); + + if (last_attempt || !is_retriable) + { + if (!mute_logging) + LOG_ERROR(log, + "Failed to make request to '{}'. Error: '{}'. " + "Failed at try {}/{}.", + initial_uri.toString(), error_message, + attempt, read_settings.http_max_tries); + + std::rethrow_exception(exception); + } else { - throw; + if (on_retry) + on_retry(); + + if (!mute_logging) + LOG_INFO(log, + "Failed to make request to `{}`. Error: {}. " + "Failed at try {}/{}. " + "Will retry with current backoff wait is {}/{} ms.", + initial_uri.toString(), error_message, + attempt + 1, read_settings.http_max_tries, + milliseconds_to_wait, read_settings.http_retry_max_backoff_ms); + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, read_settings.http_retry_max_backoff_ms); } } } -template -void ReadWriteBufferFromHTTPBase::initialize() + +std::unique_ptr ReadWriteBufferFromHTTP::initialize() { Poco::Net::HTTPResponse response; - call(session, response, method); - if (initialization_error != InitializeError::NONE) - return; + std::optional range; + if (withPartialContent()) + range = HTTPRange{getOffset(), read_range.end}; - while (isRedirect(response.getStatus())) - { - Poco::URI uri_redirect = getUriAfterRedirect(saved_uri_redirect.value_or(uri), response); - if (remote_host_filter) - remote_host_filter->checkURL(uri_redirect); + auto result = callWithRedirects(response, method, range); - session->updateSession(uri_redirect); - - istr = callImpl(session, uri_redirect, response, method); - saved_uri_redirect = uri_redirect; - } - - if (response.hasContentLength()) - LOG_DEBUG(log, "Received response with content length: {}", response.getContentLength()); - - if (withPartialContent(read_range) && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) + if (range.has_value() && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) { /// Having `200 OK` instead of `206 Partial Content` is acceptable in case we retried with range.begin == 0. if (getOffset() != 0) { - if (!exception) + /// Retry 200OK + if (response.getStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) { - exception = std::make_exception_ptr(Exception( + String reason = fmt::format( + "Cannot read with range: [{}, {}] (response status: {}, reason: {}), will retry", + *read_range.begin, read_range.end ? toString(*read_range.end) : "-", + toString(response.getStatus()), response.getReason()); + + /// it is retriable error + throw HTTPException( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + current_uri.toString(), + Poco::Net::HTTPResponse::HTTP_REQUESTED_RANGE_NOT_SATISFIABLE, + reason, + ""); + } + else + throw Exception( ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, "Cannot read with range: [{}, {}] (response status: {}, reason: {})", *read_range.begin, read_range.end ? toString(*read_range.end) : "-", - toString(response.getStatus()), response.getReason())); - } - - /// Retry 200OK - if (response.getStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) - initialization_error = InitializeError::RETRYABLE_ERROR; - else - initialization_error = InitializeError::NON_RETRYABLE_ERROR; - - return; + toString(response.getStatus()), response.getReason()); } else if (read_range.end) { @@ -425,257 +418,140 @@ void ReadWriteBufferFromHTTPBase::initialize() } } + response.getCookies(cookies); + content_encoding = response.get("Content-Encoding", ""); + // Remember file size. It'll be used to report eof in next nextImpl() call. if (!read_range.end && response.hasContentLength()) - file_info = parseFileInfo(response, withPartialContent(read_range) ? getOffset() : 0); + file_info = parseFileInfo(response, range.has_value() ? getOffset() : 0); - impl = std::make_unique(*istr, buffer_size); - - if (use_external_buffer) - setupExternalBuffer(); + return std::move(result).transformToReadBuffer(use_external_buffer ? 0 : buffer_size); } -template -bool ReadWriteBufferFromHTTPBase::nextImpl() +bool ReadWriteBufferFromHTTP::nextImpl() { - if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) - return false; - assert(initialization_error == InitializeError::NONE); - if (next_callback) next_callback(count()); - if ((read_range.end && getOffset() > read_range.end.value()) || - (file_info && file_info->file_size && getOffset() >= file_info->file_size.value())) - { - /// Response was fully read. - markSessionForReuse(session->getSession()); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); - return false; - } + bool next_result = false; - if (impl) - { - if (use_external_buffer) - { - setupExternalBuffer(); - } - else - { - /** - * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read. - */ - if (!working_buffer.empty()) - impl->position() = position(); - } - } - - bool result = false; - size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; - bool last_attempt = false; - - auto on_retriable_error = [&]() - { - retry_with_range_header = true; - impl.reset(); - auto http_session = session->getSession(); - http_session->reset(); - if (!last_attempt) - { - sleepForMilliseconds(milliseconds_to_wait); - milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); - } - }; - - for (size_t i = 0;; ++i) - { - if (last_attempt) - break; - last_attempt = i + 1 >= settings.http_max_tries; - - exception = nullptr; - initialization_error = InitializeError::NONE; - - try + doWithRetries( + /*callable=*/ [&] () { if (!impl) { - initialize(); - - if (initialization_error == InitializeError::NON_RETRYABLE_ERROR) + try { - assert(exception); - break; + impl = initialize(); } - else if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) + catch (HTTPException & e) { - return false; - } - else if (initialization_error == InitializeError::RETRYABLE_ERROR) - { - LOG_TRACE( - log, - "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " - "(Current backoff wait is {}/{} ms)", - uri.toString(), i + 1, settings.http_max_tries, getOffset(), - read_range.end ? toString(*read_range.end) : "unknown", - milliseconds_to_wait, settings.http_retry_max_backoff_ms); + if (http_skip_not_found_url && e.getHTTPStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND) + { + next_result = false; + return; + } - assert(exception); - on_retriable_error(); - continue; + throw; } - assert(!exception); - if (use_external_buffer) { - setupExternalBuffer(); + impl->set(internal_buffer.begin(), internal_buffer.size()); + } + else + { + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); } } - result = impl->next(); - exception = nullptr; - break; - } - catch (const Poco::Exception & e) + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + } + else + { + impl->position() = position(); + } + + next_result = impl->next(); + + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + + offset_from_begin_pos += working_buffer.size(); + }, + /*on_retry=*/ [&] () { - /// Too many open files or redirects - non-retryable. - if (e.code() == POCO_EMFILE || e.code() == ErrorCodes::TOO_MANY_REDIRECTS) - throw; + impl.reset(); + }); - /** Retry request unconditionally if nothing has been read yet. - * Otherwise if it is GET method retry with range header. - */ - bool can_retry_request = !offset_from_begin_pos || method == Poco::Net::HTTPRequest::HTTP_GET; - if (!can_retry_request) - throw; - - LOG_INFO( - log, - "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " - "Error: {}. (Current backoff wait is {}/{} ms)", - uri.toString(), - i + 1, - settings.http_max_tries, - getOffset(), - read_range.end ? toString(*read_range.end) : "unknown", - e.displayText(), - milliseconds_to_wait, - settings.http_retry_max_backoff_ms); - - on_retriable_error(); - exception = std::current_exception(); - } - } - - if (exception) - std::rethrow_exception(exception); - - if (!result) - { - /// Eof is reached, i.e response was fully read. - markSessionForReuse(session->getSession()); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); - return false; - } - - internal_buffer = impl->buffer(); - working_buffer = internal_buffer; - offset_from_begin_pos += working_buffer.size(); - return true; + return next_result; } -template -size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) +size_t ReadWriteBufferFromHTTP::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) const { /// Caller must have checked supportsReadAt(). - /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. + /// This ensures we've sent at least one HTTP request and populated current_uri. chassert(file_info && file_info->seekable); - Poco::URI uri_ = saved_uri_redirect.value_or(uri); - if (uri_.getPath().empty()) - uri_.setPath("/"); - size_t initial_n = n; - size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + size_t total_bytes_copied = 0; + size_t bytes_copied = 0; + bool is_canceled = false; - for (size_t attempt = 0; n > 0; ++attempt) - { - bool last_attempt = attempt + 1 >= settings.http_max_tries; - - Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - prepareRequest(request, uri_, HTTPRange { .begin = offset, .end = offset + n - 1}); - - LOG_TRACE(log, "Sending request to {} for range [{}, {})", uri_.toString(), offset, offset + n); - - auto sess = session->createDetachedSession(uri_); - - Poco::Net::HTTPResponse response; - std::istream * result_istr; - size_t bytes_copied = 0; - - try + doWithRetries( + /*callable=*/ [&] () { - sess->sendRequest(request); - result_istr = receiveResponse(*sess, request, response, /*allow_redirects*/ false); + auto range = HTTPRange{offset, offset + n - 1}; + + Poco::Net::HTTPResponse response; + auto result = callImpl(response, current_uri, method, range, false); if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT && (offset != 0 || offset + n < *file_info->file_size)) - throw Exception( - ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, - "Expected 206 Partial Content, got {} when reading {} range [{}, {})", - toString(response.getStatus()), uri_.toString(), offset, offset + n); - - copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &bytes_copied); - if (bytes_copied == n) { - result_istr->ignore(UINT64_MAX); - /// Response was fully read. - markSessionForReuse(*sess); - ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); + String reason = fmt::format( + "When reading with readBigAt {}." + "Cannot read with range: [{}, {}] (response status: {}, reason: {}), will retry", + initial_uri.toString(), + *range.begin, *range.end, + toString(response.getStatus()), response.getReason()); + + throw HTTPException( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + current_uri.toString(), + Poco::Net::HTTPResponse::HTTP_REQUESTED_RANGE_NOT_SATISFIABLE, + reason, + ""); } - } - catch (const Poco::Exception & e) + + copyFromIStreamWithProgressCallback(*result.response_stream, to, n, progress_callback, &bytes_copied, &is_canceled); + + offset += bytes_copied; + total_bytes_copied += bytes_copied; + to += bytes_copied; + n -= bytes_copied; + bytes_copied = 0; + }, + /*on_retry=*/ [&] () { - LOG_ERROR( - log, - "HTTP request (positioned) to `{}` with range [{}, {}) failed at try {}/{}: {}", - uri_.toString(), offset, offset + n, attempt + 1, settings.http_max_tries, - e.what()); + offset += bytes_copied; + total_bytes_copied += bytes_copied; + to += bytes_copied; + n -= bytes_copied; + bytes_copied = 0; + }); - /// Decide whether to retry. - - if (last_attempt) - throw; - - /// Too many open files - non-retryable. - if (e.code() == POCO_EMFILE) - throw; - - if (const auto * h = dynamic_cast(&e); - h && !isRetriableError(static_cast(h->getHTTPStatus()))) - throw; - - sleepForMilliseconds(milliseconds_to_wait); - milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); - } - - /// Make sure retries don't re-read the bytes that we've already reported to progress_callback. - offset += bytes_copied; - to += bytes_copied; - n -= bytes_copied; - } - - return initial_n; + chassert(total_bytes_copied == initial_n || is_canceled); + return total_bytes_copied; } -template -off_t ReadWriteBufferFromHTTPBase::getPosition() { return getOffset() - available(); } +off_t ReadWriteBufferFromHTTP::getPosition() +{ + return getOffset() - available(); +} -template -off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int whence) +off_t ReadWriteBufferFromHTTP::seek(off_t offset_, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET mode is allowed."); @@ -688,8 +564,8 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (!working_buffer.empty() && size_t(offset_) >= current_offset - working_buffer.size() && offset_ < current_offset) { pos = working_buffer.end() - (current_offset - offset_); - assert(pos >= working_buffer.begin()); - assert(pos < working_buffer.end()); + chassert(pos >= working_buffer.begin()); + chassert(pos < working_buffer.end()); return getPosition(); } @@ -700,7 +576,7 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (offset_ > position) { size_t diff = offset_ - position; - if (diff < settings.remote_read_min_bytes_for_seek) + if (diff < read_settings.remote_read_min_bytes_for_seek) { ignore(diff); return offset_; @@ -709,6 +585,7 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int if (!atEndOfRequestedRangeGuess()) ProfileEvents::increment(ProfileEvents::ReadBufferSeekCancelConnection); + impl.reset(); } @@ -719,8 +596,8 @@ off_t ReadWriteBufferFromHTTPBase::seek(off_t offset_, int return offset_; } -template -void ReadWriteBufferFromHTTPBase::setReadUntilPosition(size_t until) + +void ReadWriteBufferFromHTTP::setReadUntilPosition(size_t until) { until = std::max(until, 1ul); if (read_range.end && *read_range.end + 1 == until) @@ -736,8 +613,7 @@ void ReadWriteBufferFromHTTPBase::setReadUntilPosition(size } } -template -void ReadWriteBufferFromHTTPBase::setReadUntilEnd() +void ReadWriteBufferFromHTTP::setReadUntilEnd() { if (!read_range.end) return; @@ -752,11 +628,9 @@ void ReadWriteBufferFromHTTPBase::setReadUntilEnd() } } -template -bool ReadWriteBufferFromHTTPBase::supportsRightBoundedReads() const { return true; } +bool ReadWriteBufferFromHTTP::supportsRightBoundedReads() const { return true; } -template -bool ReadWriteBufferFromHTTPBase::atEndOfRequestedRangeGuess() +bool ReadWriteBufferFromHTTP::atEndOfRequestedRangeGuess() { if (!impl) return true; @@ -767,8 +641,7 @@ bool ReadWriteBufferFromHTTPBase::atEndOfRequestedRangeGues return false; } -template -std::string ReadWriteBufferFromHTTPBase::getResponseCookie(const std::string & name, const std::string & def) const +std::string ReadWriteBufferFromHTTP::getResponseCookie(const std::string & name, const std::string & def) const { for (const auto & cookie : cookies) if (cookie.getName() == name) @@ -776,19 +649,19 @@ std::string ReadWriteBufferFromHTTPBase::getResponseCookie( return def; } -template -void ReadWriteBufferFromHTTPBase::setNextCallback(NextCallback next_callback_) +void ReadWriteBufferFromHTTP::setNextCallback(NextCallback next_callback_) { next_callback = next_callback_; /// Some data maybe already read next_callback(count()); } -template -const std::string & ReadWriteBufferFromHTTPBase::getCompressionMethod() const { return content_encoding; } +const std::string & ReadWriteBufferFromHTTP::getCompressionMethod() const +{ + return content_encoding; +} -template -std::optional ReadWriteBufferFromHTTPBase::tryGetLastModificationTime() +std::optional ReadWriteBufferFromHTTP::tryGetLastModificationTime() { if (!file_info) { @@ -805,12 +678,11 @@ std::optional ReadWriteBufferFromHTTPBase::tryGetLa return file_info->last_modified; } -template -HTTPFileInfo ReadWriteBufferFromHTTPBase::getFileInfo() +ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() { /// May be disabled in case the user knows in advance that the server doesn't support HEAD requests. /// Allows to avoid making unnecessary requests in such cases. - if (!settings.http_make_head_request) + if (!read_settings.http_make_head_request) return HTTPFileInfo{}; Poco::Net::HTTPResponse response; @@ -832,11 +704,11 @@ HTTPFileInfo ReadWriteBufferFromHTTPBase::getFileInfo() throw; } + return parseFileInfo(response, 0); } -template -HTTPFileInfo ReadWriteBufferFromHTTPBase::parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin) +ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin) { HTTPFileInfo res; @@ -869,78 +741,3 @@ HTTPFileInfo ReadWriteBufferFromHTTPBase::parseFileInfo(con } -SessionFactory::SessionFactory(const ConnectionTimeouts & timeouts_, ProxyConfiguration proxy_config_) - : timeouts(timeouts_), proxy_config(proxy_config_) {} - -SessionFactory::SessionType SessionFactory::buildNewSession(const Poco::URI & uri) -{ - return makeHTTPSession(uri, timeouts, proxy_config); -} - -ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, - const ConnectionTimeouts & timeouts, - const Poco::Net::HTTPBasicCredentials & credentials_, - const UInt64 max_redirects, - size_t buffer_size_, - const ReadSettings & settings_, - const HTTPHeaderEntries & http_header_entries_, - const RemoteHostFilter * remote_host_filter_, - bool delay_initialization_, - bool use_external_buffer_, - bool skip_not_found_url_, - std::optional file_info_, - ProxyConfiguration proxy_config_) - : Parent( - std::make_shared(uri_, max_redirects, std::make_shared(timeouts, proxy_config_)), - uri_, - credentials_, - method_, - out_stream_callback_, - buffer_size_, - settings_, - http_header_entries_, - remote_host_filter_, - delay_initialization_, - use_external_buffer_, - skip_not_found_url_, - file_info_, - proxy_config_) {} - - -PooledSessionFactory::PooledSessionFactory( - const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_) - : timeouts(timeouts_) - , per_endpoint_pool_size(per_endpoint_pool_size_) {} - -PooledSessionFactory::SessionType PooledSessionFactory::buildNewSession(const Poco::URI & uri) -{ - return makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size); -} - - -PooledReadWriteBufferFromHTTP::PooledReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, - const Poco::Net::HTTPBasicCredentials & credentials_, - size_t buffer_size_, - const UInt64 max_redirects, - PooledSessionFactoryPtr session_factory) - : Parent( - std::make_shared(uri_, max_redirects, session_factory), - uri_, - credentials_, - method_, - out_stream_callback_, - buffer_size_) {} - - -template class UpdatableSession; -template class UpdatableSession; -template class detail::ReadWriteBufferFromHTTPBase>>; -template class detail::ReadWriteBufferFromHTTPBase>>; - -} diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 63ca3e0417c..5df87fb6149 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -20,7 +20,6 @@ #include #include #include -#include #include #include "config.h" #include @@ -30,44 +29,19 @@ namespace DB { -template -class UpdatableSession +class ReadWriteBufferFromHTTP : public SeekableReadBuffer, public WithFileName, public WithFileSize { public: - using SessionPtr = typename TSessionFactory::SessionType; - - explicit UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr session_factory_); - - SessionPtr getSession(); - - void updateSession(const Poco::URI & uri); - - /// Thread safe. - SessionPtr createDetachedSession(const Poco::URI & uri); - - std::shared_ptr> clone(const Poco::URI & uri); + /// Information from HTTP response header. + struct HTTPFileInfo + { + // nullopt if the server doesn't report it. + std::optional file_size; + std::optional last_modified; + bool seekable = false; + }; private: - SessionPtr session; - UInt64 redirects{0}; - UInt64 max_redirects; - Poco::URI initial_uri; - std::shared_ptr session_factory; -}; - - -/// Information from HTTP response header. -struct HTTPFileInfo -{ - // nullopt if the server doesn't report it. - std::optional file_size; - std::optional last_modified; - bool seekable = false; -}; - - -namespace detail -{ /// Byte range, including right bound [begin, end]. struct HTTPRange { @@ -75,218 +49,208 @@ namespace detail std::optional end; }; - template - class ReadWriteBufferFromHTTPBase : public SeekableReadBuffer, public WithFileName, public WithFileSize + struct CallResult { - protected: - Poco::URI uri; - std::string method; - std::string content_encoding; + HTTPSessionPtr session; + std::istream * response_stream = nullptr; - UpdatableSessionPtr session; - std::istream * istr; /// owned by session - std::unique_ptr impl; - std::function out_stream_callback; - const Poco::Net::HTTPBasicCredentials & credentials; - std::vector cookies; - HTTPHeaderEntries http_header_entries; - const RemoteHostFilter * remote_host_filter = nullptr; - std::function next_callback; + CallResult(HTTPSessionPtr && session_, std::istream & response_stream_) + : session(session_) + , response_stream(&response_stream_) + {} + CallResult(CallResult &&) = default; + CallResult & operator= (CallResult &&) = default; - size_t buffer_size; - bool use_external_buffer; - - size_t offset_from_begin_pos = 0; - HTTPRange read_range; - std::optional file_info; - - /// Delayed exception in case retries with partial content are not satisfiable. - std::exception_ptr exception; - bool retry_with_range_header = false; - /// In case of redirects, save result uri to use it if we retry the request. - std::optional saved_uri_redirect; - - bool http_skip_not_found_url; - - ReadSettings settings; - LoggerPtr log; - - ProxyConfiguration proxy_config; - - bool withPartialContent(const HTTPRange & range) const; - - size_t getOffset() const; - - void prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const; - - std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info = false); - - size_t getFileSize() override; - - bool supportsReadAt() override; - - bool checkIfActuallySeekable() override; - - String getFileName() const override; - - enum class InitializeError - { - RETRYABLE_ERROR, - /// If error is not retriable, `exception` variable must be set. - NON_RETRYABLE_ERROR, - /// Allows to skip not found urls for globs - SKIP_NOT_FOUND_URL, - NONE, - }; - - InitializeError initialization_error = InitializeError::NONE; - - private: - void getHeadResponse(Poco::Net::HTTPResponse & response); - - void setupExternalBuffer(); - - public: - using NextCallback = std::function; - using OutStreamCallback = std::function; - - explicit ReadWriteBufferFromHTTPBase( - UpdatableSessionPtr session_, - Poco::URI uri_, - const Poco::Net::HTTPBasicCredentials & credentials_, - const std::string & method_ = {}, - OutStreamCallback out_stream_callback_ = {}, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - const ReadSettings & settings_ = {}, - HTTPHeaderEntries http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization = false, - bool use_external_buffer_ = false, - bool http_skip_not_found_url_ = false, - std::optional file_info_ = std::nullopt, - ProxyConfiguration proxy_config_ = {}); - - void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false); - - void call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false); - - /** - * Throws if error is retryable, otherwise sets initialization_error = NON_RETRYABLE_ERROR and - * saves exception into `exception` variable. In case url is not found and skip_not_found_url == true, - * sets initialization_error = SKIP_NOT_FOUND_URL, otherwise throws. - */ - void initialize(); - - bool nextImpl() override; - - size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; - - off_t getPosition() override; - - off_t seek(off_t offset_, int whence) override; - - void setReadUntilPosition(size_t until) override; - - void setReadUntilEnd() override; - - bool supportsRightBoundedReads() const override; - - // If true, if we destroy impl now, no work was wasted. Just for metrics. - bool atEndOfRequestedRangeGuess(); - - std::string getResponseCookie(const std::string & name, const std::string & def) const; - - /// Set function to call on each nextImpl, useful when you need to track - /// progress. - /// NOTE: parameter on each call is not incremental -- it's all bytes count - /// passed through the buffer - void setNextCallback(NextCallback next_callback_); - - const std::string & getCompressionMethod() const; - - std::optional tryGetLastModificationTime(); - - HTTPFileInfo getFileInfo(); - - HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); + std::unique_ptr transformToReadBuffer(size_t buf_size) &&; }; -} -class SessionFactory -{ -public: - explicit SessionFactory(const ConnectionTimeouts & timeouts_, ProxyConfiguration proxy_config_ = {}); + const HTTPConnectionGroupType connection_group; + const Poco::URI initial_uri; + const std::string method; + const ProxyConfiguration proxy_config; + const ReadSettings read_settings; + const ConnectionTimeouts timeouts; - using SessionType = HTTPSessionPtr; + const Poco::Net::HTTPBasicCredentials & credentials; + const RemoteHostFilter * remote_host_filter; - SessionType buildNewSession(const Poco::URI & uri); -private: - ConnectionTimeouts timeouts; - ProxyConfiguration proxy_config; -}; + const size_t buffer_size; + const size_t max_redirects; -class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> -{ - using SessionType = UpdatableSession; - using Parent = detail::ReadWriteBufferFromHTTPBase>; + const bool use_external_buffer; + const bool http_skip_not_found_url; + + std::function out_stream_callback; + + Poco::URI current_uri; + size_t redirects = 0; + + std::string content_encoding; + std::unique_ptr impl; + + std::vector cookies; + HTTPHeaderEntries http_header_entries; + std::function next_callback; + + size_t offset_from_begin_pos = 0; + HTTPRange read_range; + std::optional file_info; + + LoggerPtr log; + + bool withPartialContent() const; + + void prepareRequest(Poco::Net::HTTPRequest & request, std::optional range) const; + + void doWithRetries(std::function && callable, std::function on_retry = nullptr, bool mute_logging = false) const; + + CallResult callImpl( + Poco::Net::HTTPResponse & response, + const Poco::URI & uri_, + const std::string & method_, + const std::optional & range, + bool allow_redirects) const; + + CallResult callWithRedirects( + Poco::Net::HTTPResponse & response, + const String & method_, + const std::optional & range); + + std::unique_ptr initialize(); + + size_t getFileSize() override; + + bool supportsReadAt() override; + + bool checkIfActuallySeekable() override; + + String getFileName() const override; + + void getHeadResponse(Poco::Net::HTTPResponse & response); + + void setupExternalBuffer(); + + size_t getOffset() const; + + // If true, if we destroy impl now, no work was wasted. Just for metrics. + bool atEndOfRequestedRangeGuess(); public: + using NextCallback = std::function; + using OutStreamCallback = std::function; + ReadWriteBufferFromHTTP( - Poco::URI uri_, + const HTTPConnectionGroupType & connection_group_, + const Poco::URI & uri_, const std::string & method_, - OutStreamCallback out_stream_callback_, - const ConnectionTimeouts & timeouts, - const Poco::Net::HTTPBasicCredentials & credentials_, - const UInt64 max_redirects = 0, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - const ReadSettings & settings_ = {}, - const HTTPHeaderEntries & http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization_ = true, - bool use_external_buffer_ = false, - bool skip_not_found_url_ = false, - std::optional file_info_ = std::nullopt, - ProxyConfiguration proxy_config_ = {}); -}; - -class PooledSessionFactory -{ -public: - explicit PooledSessionFactory( - const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_); - - using SessionType = PooledHTTPSessionPtr; - - /// Thread safe. - SessionType buildNewSession(const Poco::URI & uri); - -private: - ConnectionTimeouts timeouts; - size_t per_endpoint_pool_size; -}; - -using PooledSessionFactoryPtr = std::shared_ptr; - -class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> -{ - using SessionType = UpdatableSession; - using Parent = detail::ReadWriteBufferFromHTTPBase>; - -public: - explicit PooledReadWriteBufferFromHTTP( - Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback_, + ProxyConfiguration proxy_config_, + ReadSettings read_settings_, + ConnectionTimeouts timeouts_, const Poco::Net::HTTPBasicCredentials & credentials_, + const RemoteHostFilter * remote_host_filter_, size_t buffer_size_, - const UInt64 max_redirects, - PooledSessionFactoryPtr session_factory); + size_t max_redirects_, + OutStreamCallback out_stream_callback_, + bool use_external_buffer_, + bool http_skip_not_found_url_, + HTTPHeaderEntries http_header_entries_, + bool delay_initialization, + std::optional file_info_); + + bool nextImpl() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) const override; + + off_t seek(off_t offset_, int whence) override; + + void setReadUntilPosition(size_t until) override; + + void setReadUntilEnd() override; + + bool supportsRightBoundedReads() const override; + + off_t getPosition() override; + + std::string getResponseCookie(const std::string & name, const std::string & def) const; + + /// Set function to call on each nextImpl, useful when you need to track + /// progress. + /// NOTE: parameter on each call is not incremental -- it's all bytes count + /// passed through the buffer + void setNextCallback(NextCallback next_callback_); + + const std::string & getCompressionMethod() const; + + std::optional tryGetLastModificationTime(); + + HTTPFileInfo getFileInfo(); + static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); }; +using ReadWriteBufferFromHTTPPtr = std::unique_ptr; -extern template class UpdatableSession; -extern template class UpdatableSession; -extern template class detail::ReadWriteBufferFromHTTPBase>>; -extern template class detail::ReadWriteBufferFromHTTPBase>>; +class BuilderRWBufferFromHTTP +{ + Poco::URI uri; + std::string method = Poco::Net::HTTPRequest::HTTP_GET; + HTTPConnectionGroupType connection_group = HTTPConnectionGroupType::HTTP; + ProxyConfiguration proxy_config{}; + ReadSettings read_settings{}; + ConnectionTimeouts timeouts{}; + const RemoteHostFilter * remote_host_filter = nullptr; + size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE; + size_t max_redirects = 0; + ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = nullptr; + bool use_external_buffer = false; + HTTPHeaderEntries http_header_entries{}; + bool delay_initialization = true; + +public: + BuilderRWBufferFromHTTP(Poco::URI uri_) + : uri(uri_) + {} + +#define setterMember(name, member) \ + BuilderRWBufferFromHTTP & name(decltype(BuilderRWBufferFromHTTP::member) arg_##member) \ + { \ + member = std::move(arg_##member); \ + return *this; \ + } + + setterMember(withConnectionGroup, connection_group) + setterMember(withMethod, method) + setterMember(withProxy, proxy_config) + setterMember(withSettings, read_settings) + setterMember(withTimeouts, timeouts) + setterMember(withHostFilter, remote_host_filter) + setterMember(withBufSize, buffer_size) + setterMember(withRedirects, max_redirects) + setterMember(withOutCallback, out_stream_callback) + setterMember(withHeaders, http_header_entries) + setterMember(withExternalBuf, use_external_buffer) + setterMember(withDelayInit, delay_initialization) +#undef setterMember + + ReadWriteBufferFromHTTPPtr create(const Poco::Net::HTTPBasicCredentials & credentials_) + { + return std::make_unique( + connection_group, + uri, + method, + proxy_config, + read_settings, + timeouts, + credentials_, + remote_host_filter, + buffer_size, + max_redirects, + out_stream_callback, + use_external_buffer, + /*http_skip_not_found_url=*/ false, + http_header_entries, + delay_initialization, + /*file_info_=*/ std::nullopt); + } +}; } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index dbb93e63143..a29a4b0b8ee 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,5 +1,4 @@ #include -#include "Common/DNSResolver.h" #include "config.h" #if USE_AWS_S3 @@ -147,9 +146,7 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withTCPKeepAliveTimeout(Poco::Timespan( - client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)) - .withHTTPKeepAliveTimeout(Poco::Timespan( - client_configuration.http_keep_alive_timeout_ms * 1000)); /// flag indicating whether keep-alive is enabled is set to each session upon creation + client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)); } PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration) @@ -164,8 +161,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , get_request_throttler(client_configuration.get_request_throttler) , put_request_throttler(client_configuration.put_request_throttler) , extra_headers(client_configuration.extra_headers) - , http_connection_pool_size(client_configuration.http_connection_pool_size) - , wait_on_pool_size_limit(client_configuration.wait_on_pool_size_limit) { } @@ -308,12 +303,8 @@ void PocoHTTPClient::makeRequestInternal( Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { - /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. const auto request_configuration = per_request_configuration(); - if (http_connection_pool_size) - makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); - else - makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); + makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); } String getMethod(const Aws::Http::HttpRequest & request) @@ -335,7 +326,6 @@ String getMethod(const Aws::Http::HttpRequest & request) } } -template void PocoHTTPClient::makeRequestInternalImpl( Aws::Http::HttpRequest & request, const DB::ProxyConfiguration & proxy_configuration, @@ -343,8 +333,6 @@ void PocoHTTPClient::makeRequestInternalImpl( Aws::Utils::RateLimits::RateLimiterInterface *, Aws::Utils::RateLimits::RateLimiterInterface *) const { - using SessionPtr = std::conditional_t; - LoggerPtr log = getLogger("AWSClient"); auto uri = request.GetUri().GetURIString(); @@ -396,40 +384,17 @@ void PocoHTTPClient::makeRequestInternalImpl( for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt) { Poco::URI target_uri(uri); - SessionPtr session; - if (!proxy_configuration.host.empty()) - { - if (enable_s3_requests_logging) - LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri); - /// Reverse proxy can replace host header with resolved ip address instead of host name. - /// This can lead to request signature difference on S3 side. - if constexpr (pooled) - session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit, - proxy_configuration); - else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - proxy_configuration); - } - else - { - if constexpr (pooled) - session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit); - else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true)); - } + if (enable_s3_requests_logging && !proxy_configuration.isEmpty()) + LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri); + + auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; + + auto session = makeHTTPSession( + group, + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + proxy_configuration); /// In case of error this address will be written to logs request.SetResolvedRemoteHost(session->getResolvedAddress()); @@ -612,10 +577,6 @@ void PocoHTTPClient::makeRequestInternalImpl( response->SetClientErrorMessage(getCurrentExceptionMessage(false)); addMetric(request, S3MetricType::Errors); - - /// Probably this is socket timeout or something more or less related to DNS - /// Let's just remove this host from DNS cache to be more safe - DNSResolver::instance().removeHostFromCache(Poco::URI(uri).getHost()); } } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 5178d75e7b6..a93a4dfbaf7 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include @@ -49,12 +49,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration ThrottlerPtr put_request_throttler; HTTPHeaderEntries extra_headers; - /// Not a client parameter in terms of HTTP and we won't send it to the server. Used internally to determine when connection have to be re-established. - uint32_t http_keep_alive_timeout_ms = 0; - /// Zero means pooling will not be used. - size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit - bool wait_on_pool_size_limit = true; bool s3_use_adaptive_timeouts = true; std::function error_report; @@ -98,12 +93,6 @@ public: ); } - void SetResponseBody(Aws::IStream & incoming_stream, PooledHTTPSessionPtr & session_) /// NOLINT - { - body_stream = Aws::Utils::Stream::ResponseStream( - Aws::New>("http result streambuf", session_, incoming_stream.rdbuf())); - } - void SetResponseBody(std::string & response_body) /// NOLINT { auto stream = Aws::New("http result buf", response_body); // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -163,7 +152,6 @@ private: EnumSize, }; - template void makeRequestInternalImpl( Aws::Http::HttpRequest & request, const DB::ProxyConfiguration & proxy_configuration, @@ -196,9 +184,6 @@ protected: ThrottlerPtr put_request_throttler; const HTTPHeaderEntries extra_headers; - - size_t http_connection_pool_size = 0; - bool wait_on_pool_size_limit = true; }; } diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index c002d30e633..798833e1a9b 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -82,7 +82,7 @@ public: /// (e.g. next() or supportsReadAt()). /// * Performance: there's no buffering. Each readBigAt() call typically translates into actual /// IO operation (e.g. HTTP request). Don't use it for small adjacent reads. - virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) + virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readBigAt() not implemented"); } /// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat). diff --git a/src/IO/S3/SessionAwareIOStream.h b/src/IO/SessionAwareIOStream.h similarity index 97% rename from src/IO/S3/SessionAwareIOStream.h rename to src/IO/SessionAwareIOStream.h index babe52545d1..2380bd0fd60 100644 --- a/src/IO/S3/SessionAwareIOStream.h +++ b/src/IO/SessionAwareIOStream.h @@ -3,7 +3,7 @@ #include -namespace DB::S3 +namespace DB { /** * Wrapper of IOStream to store response stream and corresponding HTTP session. diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 8ddcbc03b84..d54e1685017 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -7,6 +7,7 @@ namespace DB { WriteBufferFromHTTP::WriteBufferFromHTTP( + const HTTPConnectionGroupType & connection_group, const Poco::URI & uri, const std::string & method, const std::string & content_type, @@ -14,9 +15,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const HTTPHeaderEntries & additional_headers, const ConnectionTimeouts & timeouts, size_t buffer_size_, - ProxyConfiguration proxy_configuration) + ProxyConfiguration proxy_configuration +) : WriteBufferFromOStream(buffer_size_) - , session{makeHTTPSession(uri, timeouts, proxy_configuration)} + , session{makeHTTPSession(connection_group, uri, timeouts, proxy_configuration)} , request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} { request.setHost(uri.getHost()); diff --git a/src/IO/WriteBufferFromHTTP.h b/src/IO/WriteBufferFromHTTP.h index f1e1e2a9e91..09fd55ec290 100644 --- a/src/IO/WriteBufferFromHTTP.h +++ b/src/IO/WriteBufferFromHTTP.h @@ -19,7 +19,8 @@ namespace DB class WriteBufferFromHTTP : public WriteBufferFromOStream { public: - explicit WriteBufferFromHTTP(const Poco::URI & uri, + explicit WriteBufferFromHTTP(const HTTPConnectionGroupType & connection_group, + const Poco::URI & uri, const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only const std::string & content_type = "", const std::string & content_encoding = "", diff --git a/src/IO/copyData.cpp b/src/IO/copyData.cpp index 07222a930b5..d2c7200c350 100644 --- a/src/IO/copyData.cpp +++ b/src/IO/copyData.cpp @@ -35,7 +35,7 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t } if (check_bytes && bytes > 0) - throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF."); + throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF, left to copy {} bytes.", bytes); } void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function cancellation_hook, ThrottlerPtr throttler) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a81392cb3d8..d658fbe9920 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -380,8 +380,6 @@ struct ContextSharedPart : boost::noncopyable OrdinaryBackgroundExecutorPtr moves_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); - /// The global pool of HTTP sessions for background fetches. - PooledSessionFactoryPtr fetches_session_factory TSA_GUARDED_BY(background_executors_mutex); RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml HTTPHeaderFilter http_header_filter; /// Forbidden HTTP headers from config.xml @@ -5039,11 +5037,6 @@ void Context::initializeBackgroundExecutorsIfNeeded() ); LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size); - auto timeouts = ConnectionTimeouts::getFetchPartHTTPTimeouts(getServerSettings(), getSettingsRef()); - /// The number of background fetches is limited by the number of threads in the background thread pool. - /// It doesn't make any sense to limit the number of connections per host any further. - shared->fetches_session_factory = std::make_shared(timeouts, background_fetches_pool_size); - shared->fetch_executor = std::make_shared ( "Fetch", @@ -5097,12 +5090,6 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const return shared->common_executor; } -PooledSessionFactoryPtr Context::getCommonFetchesSessionFactory() const -{ - SharedLockGuard lock(shared->background_executors_mutex); - return shared->fetches_session_factory; -} - IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const { callOnce(shared->readers_initialized, [&] { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b2310eaa85d..c8aa3604a6f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -206,9 +206,6 @@ using TemporaryDataOnDiskScopePtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; -class PooledSessionFactory; -using PooledSessionFactoryPtr = std::shared_ptr; - class SessionTracker; struct ServerSettings; @@ -1226,7 +1223,6 @@ public: OrdinaryBackgroundExecutorPtr getMovesExecutor() const; OrdinaryBackgroundExecutorPtr getFetchesExecutor() const; OrdinaryBackgroundExecutorPtr getCommonExecutor() const; - PooledSessionFactoryPtr getCommonFetchesSessionFactory() const; IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; #if USE_LIBURING diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 6a8f82914bf..fe2baea6b4e 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -44,6 +44,11 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const { return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port); } + catch (const DB::NetException &) + { + /// Avoid "Host not found" exceptions + return false; + } catch (const Poco::Net::NetException &) { /// Avoid "Host not found" exceptions diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 37f3c8b2958..026e0c166b4 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -333,10 +334,17 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE); DNSResolver::instance().dropCache(); + HostResolversPool::instance().dropCache(); /// Reinitialize clusters to update their resolved_addresses system_context->reloadClusterConfig(); break; } + case Type::DROP_CONNECTIONS_CACHE: + { + getContext()->checkAccess(AccessType::SYSTEM_DROP_CONNECTIONS_CACHE); + HTTPConnectionPools::instance().dropCache(); + break; + } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context->clearMarkCache(); @@ -1201,6 +1209,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() break; } case Type::DROP_DNS_CACHE: + case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MARK_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index e2ebaee8438..effc7207793 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -384,6 +384,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::KILL: case Type::SHUTDOWN: case Type::DROP_DNS_CACHE: + case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: case Type::DROP_MARK_CACHE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 48be7f6b84f..70a9e27178d 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -22,6 +22,7 @@ public: KILL, SUSPEND, DROP_DNS_CACHE, + DROP_CONNECTIONS_CACHE, DROP_MARK_CACHE, DROP_UNCOMPRESSED_CACHE, DROP_INDEX_MARK_CACHE, diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 8ef2cda5587..2ed55cca30c 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1016,7 +1016,7 @@ private: http_basic_credentials.authenticate(request); } - auto session = makePooledHTTPSession(url, timeouts, 1); + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, url, timeouts); session->sendRequest(request); Poco::Net::HTTPResponse response; @@ -1025,8 +1025,6 @@ private: Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); - /// Response was fully read. - markSessionForReuse(session); auto schema = json_body->getValue("schema"); LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 168c5f729ce..05e1129f9dc 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -526,14 +526,12 @@ std::pair Fetcher::fetchSelected creds.setPassword(password); } - std::unique_ptr in = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - nullptr, - creds, - DBMS_DEFAULT_BUFFER_SIZE, - 0, /* no redirects */ - context->getCommonFetchesSessionFactory()); + auto in = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::HTTP) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(timeouts) + .withDelayInit(false) + .create(creds); int server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); String remote_fs_metadata = parse(in->getResponseCookie("remote_fs_metadata", "")); @@ -557,11 +555,13 @@ std::pair Fetcher::fetchSelected if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { readBinary(sum_files_size, *in); + if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { IMergeTreeDataPart::TTLInfos ttl_infos; String ttl_infos_string; readBinary(ttl_infos_string, *in); + ReadBufferFromString ttl_infos_buffer(ttl_infos_string); assertString("ttl format version: 1\n", ttl_infos_buffer); ttl_infos.read(ttl_infos_buffer); @@ -609,6 +609,7 @@ std::pair Fetcher::fetchSelected } UInt64 revision = parse(in->getResponseCookie("disk_revision", "0")); + if (revision) disk->syncRevision(revision); @@ -743,7 +744,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, bool is_projection, ThrottlerPtr throttler) @@ -799,7 +800,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( void Fetcher::downloadBaseOrProjectionPartToDisk( const String & replica_path, const MutableDataPartStoragePtr & data_part_storage, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler, @@ -807,6 +808,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( { size_t files; readBinary(files, in); + LOG_DEBUG(log, "Downloading files {}", files); + std::vector> written_files; @@ -872,7 +875,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( const String & tmp_prefix, DiskPtr disk, bool to_remote_disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, size_t projections, ThrottlerPtr throttler, diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 8c15dc3cfdb..45a6cf83872 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -20,7 +20,7 @@ namespace DB { class StorageReplicatedMergeTree; -class PooledReadWriteBufferFromHTTP; +class ReadWriteBufferFromHTTP; namespace DataPartsExchange { @@ -94,7 +94,7 @@ private: void downloadBaseOrProjectionPartToDisk( const String & replica_path, const MutableDataPartStoragePtr & data_part_storage, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler, @@ -107,7 +107,7 @@ private: const String & tmp_prefix_, DiskPtr disk, bool to_remote_disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, OutputBufferGetter output_buffer_getter, size_t projections, ThrottlerPtr throttler, @@ -120,7 +120,7 @@ private: const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, bool is_projection, ThrottlerPtr throttler); @@ -131,7 +131,7 @@ private: bool to_detached, const String & tmp_prefix_, DiskPtr disk, - PooledReadWriteBufferFromHTTP & in, + ReadWriteBufferFromHTTP & in, size_t projections, MergeTreeData::DataPart::Checksums & checksums, ThrottlerPtr throttler); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4e3d8d38b0e..3b766ac8d26 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -963,7 +963,7 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata else columns_to_send = metadata_snapshot->getSampleBlockNonMaterialized().getNames(); - /// DistributedSink will not own cluster, but will own ConnectionPools of the cluster + /// DistributedSink will not own cluster return std::make_shared( local_context, *this, metadata_snapshot, cluster, insert_sync, timeout, StorageID{remote_database, remote_table}, columns_to_send); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 72bbcdd3ea8..11da394feec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1420,8 +1420,10 @@ void StorageS3::Configuration::connect(const ContextPtr & context) url.uri.getScheme()); client_configuration.endpointOverride = url.endpoint; + /// seems as we don't use it client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; + client_configuration.connectTimeoutMs = local_settings.s3_connect_timeout_ms; + auto headers = auth_settings.headers; if (!headers_from_ast.empty()) headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 039be222e7e..b539a152b69 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -461,21 +461,23 @@ std::pair> StorageURLSource: try { auto res = std::make_unique( + HTTPConnectionGroupType::STORAGE, request_uri, http_method, - callback, + proxy_config, + read_settings, timeouts, credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, &context_->getRemoteHostFilter(), + settings.max_read_buffer_size, + settings.max_http_get_redirects, + callback, + /*use_external_buffer*/ false, + skip_url_not_found_error, + headers, delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error, - /* file_info */ std::nullopt, - proxy_config); + /*file_info_*/ std::nullopt); + if (context_->getSettingsRef().engine_url_skip_empty_files && res->eof() && option != std::prev(end)) { @@ -547,7 +549,7 @@ StorageURLSink::StorageURLSink( auto proxy_config = getProxyConfiguration(http_method); auto write_buffer = std::make_unique( - Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config + HTTPConnectionGroupType::STORAGE, Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config ); const auto & settings = context->getSettingsRef(); @@ -1320,24 +1322,17 @@ std::optional IStorageURLBase::tryGetLastModificationTime( auto proxy_config = getProxyConfiguration(uri.getScheme()); - ReadWriteBufferFromHTTP buf( - uri, - Poco::Net::HTTPRequest::HTTP_GET, - {}, - getHTTPTimeouts(context), - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - context->getReadSettings(), - headers, - &context->getRemoteHostFilter(), - true, - false, - false, - std::nullopt, - proxy_config); + auto buf = BuilderRWBufferFromHTTP(uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withSettings(context->getReadSettings()) + .withTimeouts(getHTTPTimeouts(context)) + .withHostFilter(&context->getRemoteHostFilter()) + .withBufSize(settings.max_read_buffer_size) + .withRedirects(settings.max_http_get_redirects) + .withHeaders(headers) + .create(credentials); - return buf.tryGetLastModificationTime(); + return buf->tryGetLastModificationTime(); } StorageURL::StorageURL( diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index ca6d40a05a3..a5c16b3a5aa 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -153,17 +153,16 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls)); Poco::Net::HTTPBasicCredentials credentials{}; - ReadWriteBufferFromHTTP buf( - columns_info_uri, - Poco::Net::HTTPRequest::HTTP_POST, - {}, - ConnectionTimeouts::getHTTPTimeouts( - context->getSettingsRef(), - context->getServerSettings().keep_alive_timeout), - credentials); + auto buf = BuilderRWBufferFromHTTP(columns_info_uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withTimeouts(ConnectionTimeouts::getHTTPTimeouts( + context->getSettingsRef(), + context->getServerSettings().keep_alive_timeout)) + .create(credentials); std::string columns_info; - readStringBinary(columns_info, buf); + readStringBinary(columns_info, *buf); NamesAndTypesList columns = NamesAndTypesList::parse(columns_info); return ColumnsDescription{columns}; diff --git a/tests/integration/test_backup_restore_new/test_cancel_backup.py b/tests/integration/test_backup_restore_new/test_cancel_backup.py index 6016bac9197..cce23a7e932 100644 --- a/tests/integration/test_backup_restore_new/test_cancel_backup.py +++ b/tests/integration/test_backup_restore_new/test_cancel_backup.py @@ -177,7 +177,7 @@ def cancel_restore(restore_id): def test_cancel_backup(): # We use partitioning so backups would contain more files. node.query( - "CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%5" + "CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%20" ) node.query(f"INSERT INTO tbl SELECT number FROM numbers(500)") diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml index 23ab57f9330..d94ef68d9c4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -5,6 +5,7 @@ 1 1 + 10000 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 7b1f503ed55..84f7f9f1b6d 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -19,6 +19,7 @@ minio minio123 1 + 10000 diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index e84209a03a1..dbcd7cc3c21 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -278,7 +278,7 @@ def test_unavailable_server(cluster): "Caught exception while loading metadata.*Connection refused" ) assert node2.contains_in_log( - "HTTP request to \`http://nginx:8080/test1/.*\` failed at try 1/10 with bytes read: 0/unknown. Error: Connection refused." + "Failed to make request to 'http://nginx:8080/test1/.*'. Error: 'Connection refused'. Failed at try 10/10." ) finally: node2.exec_in_container( diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index 9c1c9797383..a6db26c8575 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -46,6 +46,7 @@ def cluster_without_dns_cache_update(): except Exception as ex: print(ex) + raise finally: cluster.shutdown() @@ -61,6 +62,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") + node2.query("SYSTEM DROP CONNECTIONS CACHE") # First we check, that normal replication works node1.query( @@ -86,6 +88,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") + node2.query("SYSTEM DROP CONNECTIONS CACHE") # Data is downloaded assert_eq_with_retry(node2, "SELECT count(*) from test_table_drop", "6") @@ -124,6 +127,7 @@ def cluster_with_dns_cache_update(): except Exception as ex: print(ex) + raise finally: cluster.shutdown() @@ -267,6 +271,11 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): privileged=True, user="root", ) + node.exec_in_container( + ["bash", "-c", 'clickhouse client -q "SYSTEM DROP CONNECTIONS CACHE"'], + privileged=True, + user="root", + ) retry_count = 1 assert_eq_with_retry( @@ -296,7 +305,8 @@ def test_host_is_drop_from_cache_after_consecutive_failures( # Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back) # dns_update_short -> dns_max_consecutive_failures set to 6 assert node4.wait_for_log_line( - "Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist." + regexp="Code: 198. DB::NetException: Not found address of host: InvalidHostThatDoesNotExist.", + look_behind_lines=300, ) assert node4.wait_for_log_line( "Cached hosts not found:.*InvalidHostThatDoesNotExist**", diff --git a/tests/integration/test_http_failover/test.py b/tests/integration/test_http_failover/test.py index 41b55ef635c..5920fd980ce 100644 --- a/tests/integration/test_http_failover/test.py +++ b/tests/integration/test_http_failover/test.py @@ -56,9 +56,10 @@ def dst_node_addrs(started_cluster, request): yield - # Clear static DNS entries + # Clear static DNS entries and all keep alive connections src_node.set_hosts([]) src_node.query("SYSTEM DROP DNS CACHE") + src_node.query("SYSTEM DROP CONNECTIONS CACHE") @pytest.mark.parametrize( @@ -77,7 +78,8 @@ def dst_node_addrs(started_cluster, request): def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation): with expectation: result = src_node.query( - "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')", + settings={"http_max_tries": "3"}, ) assert result == "42\n" diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index 17a9a03008e..033f02d7bde 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -150,7 +150,7 @@ def test_url_reconnect(started_cluster): def select(): global result result = node1.query( - "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries = 10, http_retry_max_backoff_ms=1000" + "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries=10, http_retry_max_backoff_ms=1000, http_make_head_request=false" ) assert int(result) == 6581218782194912115 diff --git a/tests/integration/test_s3_table_functions/test.py b/tests/integration/test_s3_table_functions/test.py index a6def175136..ff62d1a9eac 100644 --- a/tests/integration/test_s3_table_functions/test.py +++ b/tests/integration/test_s3_table_functions/test.py @@ -80,6 +80,7 @@ def test_s3_table_functions_timeouts(started_cluster): Test with timeout limit of 1200ms. This should raise an Exception and pass. """ + with PartitionManager() as pm: pm.add_network_delay(node, 1200) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 5ef781bdc9e..70d49b7c1b0 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -118,5 +118,5 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.wfile.write(b"OK") -httpd = http.server.HTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) +httpd = http.server.ThreadingHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) httpd.serve_forever() diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index dc0fdd1a71d..931d18a3f80 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -12,6 +12,7 @@ import urllib.request import subprocess from io import StringIO from http.server import BaseHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn def is_ipv6(host): @@ -145,11 +146,19 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer) t = threading.Thread(target=httpd.serve_forever) return t, httpd diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 7af299c6728..b18ae8a99be 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -105,6 +105,7 @@ NAMED COLLECTION ADMIN ['NAMED COLLECTION CONTROL'] NAMED_COLLECTION ALL SET DEFINER [] USER_NAME ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MMAP CACHE ['SYSTEM DROP MMAP','DROP MMAP CACHE','DROP MMAP'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index f053387d1c5..187f55697e4 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -3,6 +3,7 @@ connect_timeout Seconds 10 connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_secure_ms Milliseconds 3000 external_storage_connect_timeout_sec UInt64 10 +s3_connect_timeout_ms UInt64 1000 filesystem_prefetch_max_memory_usage UInt64 1073741824 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.python b/tests/queries/0_stateless/02205_HTTP_user_agent.python index d8f8a32b6db..83089741bf2 100644 --- a/tests/queries/0_stateless/02205_HTTP_user_agent.python +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.python @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from http.server import SimpleHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn import socket import sys import threading @@ -116,11 +117,19 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(requests_amount): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) def real_func(): for i in range(requests_amount): diff --git a/tests/queries/0_stateless/02233_HTTP_ranged.python b/tests/queries/0_stateless/02233_HTTP_ranged.python index 66ef3304098..5d06e4824b1 100644 --- a/tests/queries/0_stateless/02233_HTTP_ranged.python +++ b/tests/queries/0_stateless/02233_HTTP_ranged.python @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from http.server import BaseHTTPRequestHandler, HTTPServer +from socketserver import ThreadingMixIn import socket import sys import re @@ -206,13 +207,22 @@ class HTTPServerV6(HTTPServer): address_family = socket.AF_INET6 +class ThreadedHTTPServer(ThreadingMixIn, HTTPServer): + pass + + +class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6): + pass + + def start_server(): if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) t = threading.Thread(target=httpd.serve_forever) + t.start() return t, httpd @@ -235,8 +245,6 @@ def run_test(allow_range, settings, check_retries=False): HttpProcessor.responses_to_get = ["500", "200", "206"] retries_num = len(HttpProcessor.responses_to_get) - t, httpd = start_server() - t.start() test_select(settings) download_buffer_size = settings["max_download_buffer_size"] @@ -261,12 +269,12 @@ def run_test(allow_range, settings, check_retries=False): if HttpProcessor.range_used: raise Exception("HTTP Range used while not supported") - httpd.shutdown() - t.join() print("PASSED") def main(): + t, httpd = start_server() + settings = {"max_download_buffer_size": 20} # Test Accept-Ranges=False @@ -285,10 +293,15 @@ def main(): settings["max_download_threads"] = 2 run_test(allow_range=True, settings=settings, check_retries=True) + httpd.shutdown() + t.join() + if __name__ == "__main__": try: main() + sys.stdout.flush() + os._exit(0) except Exception as ex: exc_type, exc_value, exc_traceback = sys.exc_info() traceback.print_tb(exc_traceback, file=sys.stderr) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index 288f1129b53..e346d9893a7 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "INSERT TO S3" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | sort +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort echo "CHECK WITH query_log" $CLICKHOUSE_CLIENT -nq " diff --git a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh index ce90157d004..e4a1de9a2ec 100755 --- a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh +++ b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh @@ -19,8 +19,8 @@ query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -nm --query " WITH - ProfileEvents['ReadBufferFromS3ResetSessions'] AS reset, - ProfileEvents['ReadBufferFromS3PreservedSessions'] AS preserved + ProfileEvents['DiskConnectionsReset'] AS reset, + ProfileEvents['DiskConnectionsPreserved'] AS preserved SELECT preserved > reset FROM system.query_log WHERE type = 'QueryFinish' @@ -51,7 +51,7 @@ select queryID() from( " 2>&1) ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -nm --query " -SELECT ProfileEvents['ReadWriteBufferFromHTTPPreservedSessions'] > 0 +SELECT ProfileEvents['StorageConnectionsPreserved'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() diff --git a/tests/queries/0_stateless/02833_url_without_path_encoding.sh b/tests/queries/0_stateless/02833_url_without_path_encoding.sh index b71586099cf..eb845c6b45b 100755 --- a/tests/queries/0_stateless/02833_url_without_path_encoding.sh +++ b/tests/queries/0_stateless/02833_url_without_path_encoding.sh @@ -8,5 +8,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=1" # Grep 'test%2Fa.tsv' to ensure that path wasn't encoded/decoded -$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | grep -o "test%2Fa.tsv" -m1 - +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | \ + grep -o "test%2Fa.tsv" -m1 | head -n 1 From f7f1d86e667117dd51b74747c1b1ed5f9339c466 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 4 Mar 2024 14:37:17 +0100 Subject: [PATCH 14/19] fix tests test_attach_without_fetching test_replicated_merge_tree_wait_on_shutdown --- src/Common/CurrentMetrics.cpp | 2 +- src/Disks/IO/ReadBufferFromWebServer.cpp | 1 - tests/integration/test_attach_without_fetching/test.py | 1 + .../test_replicated_merge_tree_wait_on_shutdown/test.py | 4 ++++ 4 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index f43481f665b..dfbf6199361 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -274,7 +274,7 @@ M(DistrCacheUsedConnections, "Number of currently used connections to Distributed Cache") \ M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ - M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") + M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") \ \ M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index 7509aa81d75..03300cc0714 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -114,7 +114,6 @@ bool ReadBufferFromWebServer::nextImpl() chassert(working_buffer.begin() != nullptr); chassert(impl->buffer().begin() != nullptr); - chassert(working_buffer.begin() == impl->buffer().begin()); chassert(impl->available() == 0); diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index b430387e0f1..67352e2dcbe 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -56,6 +56,7 @@ def check_data(nodes, detached_parts): node.query_with_retry("SYSTEM SYNC REPLICA test") + for node in nodes: print("> Checking data integrity for", node.name) for i in range(10): diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 67dd03098e9..995afedf415 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -40,6 +40,10 @@ def test_shutdown_and_wait(start_cluster): f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()" ) + # we stop merges on node1 to make node2 fetch all 51 origin parts from node1 + # and not to fetch a smaller set of merged covering parts + node1.query("SYSTEM STOP MERGES test_table") + node1.query("INSERT INTO test_table VALUES (0)") node2.query("SYSTEM SYNC REPLICA test_table") From 4df406d3adce0ae1fb55d742cf59ddd928e96ddb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Mar 2024 13:56:51 +0100 Subject: [PATCH 15/19] work with review notes --- src/Client/Connection.cpp | 9 ++++---- src/Common/HTTPConnectionPool.cpp | 2 +- src/Common/HostResolvePool.cpp | 13 +++++++---- src/Common/HostResolvePool.h | 2 -- src/Common/ProfileEvents.cpp | 6 ++--- src/Common/tests/gtest_connection_pool.cpp | 27 ++++++++++++++++++++++ 6 files changed, 45 insertions(+), 14 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a11a1243957..180942e6b83 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -212,8 +212,9 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); + /// Add server address to exception. Exception will preserve stack trace. + e.addMessage("({})", getDescription()); + throw; } catch (Poco::Net::NetException & e) { @@ -222,7 +223,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + /// Add server address to exception. Also Exception will remember new stack trace. It's a pity that more precise exception type is lost. throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription()); } catch (Poco::TimeoutException & e) @@ -232,7 +233,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// Remove this possible stale entry from cache DNSResolver::instance().removeHostFromCache(host); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + /// Add server address to exception. Also Exception will remember new stack trace. It's a pity that more precise exception type is lost. /// This exception can only be thrown from socket->connect(), so add information about connection timeout. const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; throw NetException( diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 18ffef34091..a21438a11a2 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -336,7 +336,7 @@ private: request_stream_completed = false; response_stream = nullptr; - response_stream_completed = true; + response_stream_completed = false; return result; } diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp index f6cc9c919ba..6db28edc07e 100644 --- a/src/Common/HostResolvePool.cpp +++ b/src/Common/HostResolvePool.cpp @@ -13,7 +13,7 @@ namespace ProfileEvents { extern const Event AddressesDiscovered; extern const Event AddressesExpired; - extern const Event AddressesFailScored; + extern const Event AddressesMarkedAsFailed; } namespace CurrentMetrics @@ -34,7 +34,7 @@ HostResolverMetrics HostResolver::getMetrics() return HostResolverMetrics{ .discovered = ProfileEvents::AddressesDiscovered, .expired = ProfileEvents::AddressesExpired, - .failed = ProfileEvents::AddressesFailScored, + .failed = ProfileEvents::AddressesMarkedAsFailed, .active_count = CurrentMetrics::AddressesActive, }; } @@ -120,7 +120,6 @@ void HostResolver::updateWeights() } chassert((getTotalWeight() > 0 && !records.empty()) || records.empty()); - random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); } HostResolver::Entry HostResolver::resolve() @@ -170,6 +169,7 @@ void HostResolver::setFail(const Poco::Net::IPAddress & address) Poco::Net::IPAddress HostResolver::selectBest() { chassert(!records.empty()); + auto random_weight_picker = std::uniform_int_distribution(0, getTotalWeight() - 1); size_t weight = random_weight_picker(thread_local_rng); auto it = std::partition_point(records.begin(), records.end(), [&](const Record & rec) { return rec.weight_prefix_sum <= weight; }); chassert(it != records.end()); @@ -178,8 +178,13 @@ Poco::Net::IPAddress HostResolver::selectBest() HostResolver::Records::iterator HostResolver::find(const Poco::Net::IPAddress & addr) TSA_REQUIRES(mutex) { - return std::lower_bound( + auto it = std::lower_bound( records.begin(), records.end(), addr, [](const Record & rec, const Poco::Net::IPAddress & value) { return rec.address < value; }); + + if (it != records.end() && it->address != addr) + return records.end(); + + return it; } bool HostResolver::isUpdateNeeded() diff --git a/src/Common/HostResolvePool.h b/src/Common/HostResolvePool.h index 2a31cec3b2d..4f127f05253 100644 --- a/src/Common/HostResolvePool.h +++ b/src/Common/HostResolvePool.h @@ -191,8 +191,6 @@ protected: Poco::Timestamp last_resolve_time TSA_GUARDED_BY(mutex); Records records TSA_GUARDED_BY(mutex); - std::uniform_int_distribution random_weight_picker TSA_GUARDED_BY(mutex); - Poco::Logger * log = &Poco::Logger::get("ConnectionPool"); }; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0c9582ab4fb..c1ac3d08245 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -721,9 +721,9 @@ The server successfully detected this situation and will download merged part fr M(HTTPConnectionsErrors, "Number of cases when creation of a http connection failed") \ M(HTTPConnectionsElapsedMicroseconds, "Total time spend on creating http connections") \ \ - M(AddressesDiscovered, "Total count of new addresses in dns resolve results for connection pools") \ - M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for for connection pools") \ - M(AddressesFailScored, "Total count of new addresses in dns resolve results for for connection pools") \ + M(AddressesDiscovered, "Total count of new addresses in dns resolve results for http connections") \ + M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for http connections") \ + M(AddressesMarkedAsFailed, "Total count of addresses which has been marked as faulty due to connection errors for http connections") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp index 01b78958442..c271cc0e2ec 100644 --- a/src/Common/tests/gtest_connection_pool.cpp +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -552,6 +552,33 @@ TEST_F(ConnectionPoolTest, HardLimit) ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); +} + +TEST_F(ConnectionPoolTest, NoReceiveCall) +{ + auto pool = getPool(); + + { + auto connection = pool->getConnection(timeouts); + + { + auto data = String("Hello"); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_PUT, "/", "HTTP/1.1"); // HTTP/1.1 is required for keep alive + request.setContentLength(data.size()); + std::ostream & ostream = connection->sendRequest(request); + ostream << data; + } + + connection->flushRequest(); + } + + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); + ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); From 21b04143e81e5e2e9a6c5fa02103bcafdb4a27ed Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Mar 2024 19:17:25 +0100 Subject: [PATCH 16/19] set vat RECORDS in main proccess --- tests/queries/0_stateless/02998_system_dns_cache_table.sh | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02998_system_dns_cache_table.sh b/tests/queries/0_stateless/02998_system_dns_cache_table.sh index 41d2386fe9c..b74fc00ab3b 100755 --- a/tests/queries/0_stateless/02998_system_dns_cache_table.sh +++ b/tests/queries/0_stateless/02998_system_dns_cache_table.sh @@ -5,12 +5,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Retries are necessary because the DNS cache may be flushed before second statement is executed -i=0 retries=3 +i=0 +retries=5 while [[ $i -lt $retries ]]; do - ${CLICKHOUSE_CURL} -sS --fail --data "SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers())" "${CLICKHOUSE_URL}" | grep -oP -q 'Ok.' && \ + ${CLICKHOUSE_CURL} -sS --fail --data "SELECT * FROM url('http://localhost:8123/ping', CSV, 'auto', headers())" "${CLICKHOUSE_URL}" | grep -oP -q 'Ok.' || continue + RECORDS=$(${CLICKHOUSE_CURL} -sS --fail --data "SELECT hostname, ip_address, ip_family, (isNotNull(cached_at) AND cached_at > '1970-01-01 00:00:00') FROM system.dns_cache WHERE hostname = 'localhost' and ip_family = 'IPv4';" "${CLICKHOUSE_URL}") - if [ "${RECORDS}" != "" ]; then + if [[ -n "${RECORDS}" ]]; then echo "${RECORDS}" exit 0 fi From 77c5de700f62451c6f2cf55620d9522832b5d56b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 9 Mar 2024 13:35:12 +0100 Subject: [PATCH 17/19] fix how web disk read empty directories --- .../ObjectStorages/Web/WebObjectStorage.cpp | 5 +-- src/IO/ReadWriteBufferFromHTTP.h | 4 ++- src/Storages/StorageURL.cpp | 32 ++++++++----------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 0bad668a404..4adb92cf5c8 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -53,6 +53,7 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lockgetReadSettings()) .withTimeouts(timeouts) .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSkipNotFound(true) .create(credentials); String file_name; @@ -98,10 +99,6 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock> StorageURLSource: try { - auto res = std::make_unique( - HTTPConnectionGroupType::STORAGE, - request_uri, - http_method, - proxy_config, - read_settings, - timeouts, - credentials, - &context_->getRemoteHostFilter(), - settings.max_read_buffer_size, - settings.max_http_get_redirects, - callback, - /*use_external_buffer*/ false, - skip_url_not_found_error, - headers, - delay_initialization, - /*file_info_*/ std::nullopt); - + auto res = BuilderRWBufferFromHTTP(request_uri) + .withConnectionGroup(HTTPConnectionGroupType::STORAGE) + .withMethod(http_method) + .withProxy(proxy_config) + .withSettings(read_settings) + .withTimeouts(timeouts) + .withHostFilter(&context_->getRemoteHostFilter()) + .withBufSize(settings.max_read_buffer_size) + .withRedirects(settings.max_http_get_redirects) + .withOutCallback(callback) + .withSkipNotFound(skip_url_not_found_error) + .withHeaders(headers) + .withDelayInit(delay_initialization) + .create(credentials); if (context_->getSettingsRef().engine_url_skip_empty_files && res->eof() && option != std::prev(end)) { From f429f54af8f1fc8fad56a96818c3fe6cc8a1ee7a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sat, 9 Mar 2024 20:38:23 +0000 Subject: [PATCH 18/19] CI: make style check fast #do_not_test --- tests/ci/style_check.py | 32 +++++++---- .../{check_cpp_docs.sh => check_cpp.sh} | 55 ++++++++++++------- utils/check-style/check_py.sh | 19 ++++--- 3 files changed, 68 insertions(+), 38 deletions(-) rename utils/check-style/{check_cpp_docs.sh => check_cpp.sh} (50%) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index d0565e136d3..a772539aef8 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,7 +10,7 @@ from pathlib import Path from typing import List, Tuple from docker_images_helper import get_docker_image, pull_image -from env_helper import REPO_COPY, TEMP_PATH +from env_helper import CI, REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -128,32 +128,40 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - # pr_info = PRInfo() + pr_info = PRInfo() IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd_1 = ( + cmd_cpp = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_cpp_docs.sh" + f"{image} ./check_cpp.sh" ) - cmd_2 = ( + cmd_py = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " f"{image} ./check_py.sh" ) - logging.info("Is going to run the command: %s", cmd_1) - logging.info("Is going to run the command: %s", cmd_2) with ProcessPoolExecutor(max_workers=2) as executor: - # Submit commands for execution in parallel - future1 = executor.submit(subprocess.run, cmd_1, shell=True) - future2 = executor.submit(subprocess.run, cmd_2, shell=True) - # Wait for both commands to complete + logging.info("Is going to run the command: %s", cmd_cpp) + future1 = executor.submit(subprocess.run, cmd_cpp, shell=True) + # Parallelization does not make it faster - run subsequently _ = future1.result() - _ = future2.result() + + run_pycheck = True + if CI and pr_info.number > 0: + # skip py check if PR and no changed py files + pr_info.fetch_changed_files() + if not any(file.endswith(".py") for file in pr_info.changed_files): + run_pycheck = False + + if run_pycheck: + logging.info("Is going to run the command: %s", cmd_py) + future2 = executor.submit(subprocess.run, cmd_py, shell=True) + _ = future2.result() # if args.push: # checkout_head(pr_info) diff --git a/utils/check-style/check_cpp_docs.sh b/utils/check-style/check_cpp.sh similarity index 50% rename from utils/check-style/check_cpp_docs.sh rename to utils/check-style/check_cpp.sh index 7ad3cede758..20d4c7f5ee9 100755 --- a/utils/check-style/check_cpp_docs.sh +++ b/utils/check-style/check_cpp.sh @@ -4,31 +4,48 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv -# FIXME: 30 sec to wait -# echo "Check duplicates" | ts -# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +start_total=`date +%s` -echo "Check style" | ts +# 40 sec - too much +# start=`date +%s` +# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +# runtime=$((`date +%s`-start)) +# echo "Duplicates check. Done. $runtime seconds." + +start=`date +%s` ./check-style -n |& tee /test_output/style_output.txt -echo "Check typos" | ts -./check-typos |& tee /test_output/typos_output.txt -echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/docs_spelling_output.txt -echo "Check whitespaces" | ts +runtime=$((`date +%s`-start)) +echo "Check style. Done. $runtime seconds." + +start=`date +%s` ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt -echo "Check workflows" | ts +runtime=$((`date +%s`-start)) +echo "Check whitespaces. Done. $runtime seconds." + +start=`date +%s` ./check-workflows |& tee /test_output/workflows_output.txt -echo "Check submodules" | ts +runtime=$((`date +%s`-start)) +echo "Check workflows. Done. $runtime seconds." + +start=`date +%s` ./check-submodules |& tee /test_output/submodules_output.txt -echo "Check style. Done" | ts +runtime=$((`date +%s`-start)) +echo "Check submodules. Done. $runtime seconds." + +start=`date +%s` +./check-typos |& tee /test_output/typos_output.txt +runtime=$((`date +%s`-start)) +echo "Check typos. Done. $runtime seconds." + +start=`date +%s` +./check-doc-aspell |& tee /test_output/docs_spelling_output.txt +runtime=$((`date +%s`-start)) +echo "Check docs spelling. Done. $runtime seconds." + +runtime=$((`date +%s`-start_total)) +echo "Check style, total. Done. $runtime seconds." + # FIXME: 6 min to wait # echo "Check shell scripts with shellcheck" | ts # ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt - - -# FIXME: move out -# /process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv -# echo "Check help for changelog generator works" | ts -# cd ../changelog || exit 1 -# ./changelog.py -h 2>/dev/null 1>&2 diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 48c02013734..5caeffa65fe 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -1,17 +1,22 @@ #!/bin/bash -# yaml check is not the best one - cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv # FIXME: 1 min to wait + head checkout # echo "Check python formatting with black" | ts # ./check-black -n |& tee /test_output/black_output.txt -echo "Check pylint" | ts -./check-pylint -n |& tee /test_output/pylint_output.txt -echo "Check pylint. Done" | ts +start_total=`date +%s` -echo "Check python type hinting with mypy" | ts +start=`date +%s` +./check-pylint -n |& tee /test_output/pylint_output.txt +runtime=$((`date +%s`-start)) +echo "Check pylint. Done. $runtime seconds." + +start=`date +%s` ./check-mypy -n |& tee /test_output/mypy_output.txt -echo "Check python type hinting with mypy. Done" | ts +runtime=$((`date +%s`-start)) +echo "Check python type hinting with mypy. Done. $runtime seconds." + +runtime=$((`date +%s`-start_total)) +echo "Check python total. Done. $runtime seconds." From 8c08fc1f579861d02690653cc05a87826e59a5b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Mar 2024 04:09:26 +0300 Subject: [PATCH 19/19] Revert "CI: make style check faster" --- tests/ci/style_check.py | 32 ++++------- .../{check_cpp.sh => check_cpp_docs.sh} | 55 +++++++------------ utils/check-style/check_py.sh | 17 ++---- 3 files changed, 37 insertions(+), 67 deletions(-) rename utils/check-style/{check_cpp.sh => check_cpp_docs.sh} (50%) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index a772539aef8..d0565e136d3 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,7 +10,7 @@ from pathlib import Path from typing import List, Tuple from docker_images_helper import get_docker_image, pull_image -from env_helper import CI, REPO_COPY, TEMP_PATH +from env_helper import REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -128,40 +128,32 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - pr_info = PRInfo() + # pr_info = PRInfo() IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd_cpp = ( + cmd_1 = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_cpp.sh" + f"{image} ./check_cpp_docs.sh" ) - cmd_py = ( + cmd_2 = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " f"--entrypoint= -w/ClickHouse/utils/check-style " f"{image} ./check_py.sh" ) + logging.info("Is going to run the command: %s", cmd_1) + logging.info("Is going to run the command: %s", cmd_2) with ProcessPoolExecutor(max_workers=2) as executor: - logging.info("Is going to run the command: %s", cmd_cpp) - future1 = executor.submit(subprocess.run, cmd_cpp, shell=True) - # Parallelization does not make it faster - run subsequently + # Submit commands for execution in parallel + future1 = executor.submit(subprocess.run, cmd_1, shell=True) + future2 = executor.submit(subprocess.run, cmd_2, shell=True) + # Wait for both commands to complete _ = future1.result() - - run_pycheck = True - if CI and pr_info.number > 0: - # skip py check if PR and no changed py files - pr_info.fetch_changed_files() - if not any(file.endswith(".py") for file in pr_info.changed_files): - run_pycheck = False - - if run_pycheck: - logging.info("Is going to run the command: %s", cmd_py) - future2 = executor.submit(subprocess.run, cmd_py, shell=True) - _ = future2.result() + _ = future2.result() # if args.push: # checkout_head(pr_info) diff --git a/utils/check-style/check_cpp.sh b/utils/check-style/check_cpp_docs.sh similarity index 50% rename from utils/check-style/check_cpp.sh rename to utils/check-style/check_cpp_docs.sh index 20d4c7f5ee9..7ad3cede758 100755 --- a/utils/check-style/check_cpp.sh +++ b/utils/check-style/check_cpp_docs.sh @@ -4,48 +4,31 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv -start_total=`date +%s` - -# 40 sec - too much -# start=`date +%s` +# FIXME: 30 sec to wait +# echo "Check duplicates" | ts # ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt -# runtime=$((`date +%s`-start)) -# echo "Duplicates check. Done. $runtime seconds." -start=`date +%s` +echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt -runtime=$((`date +%s`-start)) -echo "Check style. Done. $runtime seconds." - -start=`date +%s` -./check-whitespaces -n |& tee /test_output/whitespaces_output.txt -runtime=$((`date +%s`-start)) -echo "Check whitespaces. Done. $runtime seconds." - -start=`date +%s` -./check-workflows |& tee /test_output/workflows_output.txt -runtime=$((`date +%s`-start)) -echo "Check workflows. Done. $runtime seconds." - -start=`date +%s` -./check-submodules |& tee /test_output/submodules_output.txt -runtime=$((`date +%s`-start)) -echo "Check submodules. Done. $runtime seconds." - -start=`date +%s` +echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt -runtime=$((`date +%s`-start)) -echo "Check typos. Done. $runtime seconds." - -start=`date +%s` +echo "Check docs spelling" | ts ./check-doc-aspell |& tee /test_output/docs_spelling_output.txt -runtime=$((`date +%s`-start)) -echo "Check docs spelling. Done. $runtime seconds." - -runtime=$((`date +%s`-start_total)) -echo "Check style, total. Done. $runtime seconds." - +echo "Check whitespaces" | ts +./check-whitespaces -n |& tee /test_output/whitespaces_output.txt +echo "Check workflows" | ts +./check-workflows |& tee /test_output/workflows_output.txt +echo "Check submodules" | ts +./check-submodules |& tee /test_output/submodules_output.txt +echo "Check style. Done" | ts # FIXME: 6 min to wait # echo "Check shell scripts with shellcheck" | ts # ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt + + +# FIXME: move out +# /process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv +# echo "Check help for changelog generator works" | ts +# cd ../changelog || exit 1 +# ./changelog.py -h 2>/dev/null 1>&2 diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 5caeffa65fe..48c02013734 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -1,22 +1,17 @@ #!/bin/bash +# yaml check is not the best one + cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv # FIXME: 1 min to wait + head checkout # echo "Check python formatting with black" | ts # ./check-black -n |& tee /test_output/black_output.txt -start_total=`date +%s` - -start=`date +%s` +echo "Check pylint" | ts ./check-pylint -n |& tee /test_output/pylint_output.txt -runtime=$((`date +%s`-start)) -echo "Check pylint. Done. $runtime seconds." +echo "Check pylint. Done" | ts -start=`date +%s` +echo "Check python type hinting with mypy" | ts ./check-mypy -n |& tee /test_output/mypy_output.txt -runtime=$((`date +%s`-start)) -echo "Check python type hinting with mypy. Done. $runtime seconds." - -runtime=$((`date +%s`-start_total)) -echo "Check python total. Done. $runtime seconds." +echo "Check python type hinting with mypy. Done" | ts