From d09f5d18f16c7e988338531be2432187ce633891 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 24 Jan 2024 18:14:38 +0100 Subject: [PATCH 001/107] Repro test --- .../02967_prewhere_no_columns.reference | 2 + .../0_stateless/02967_prewhere_no_columns.sql | 51 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.reference create mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.sql diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.reference b/tests/queries/0_stateless/02967_prewhere_no_columns.reference new file mode 100644 index 00000000000..df105254618 --- /dev/null +++ b/tests/queries/0_stateless/02967_prewhere_no_columns.reference @@ -0,0 +1,2 @@ +105 +105 diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.sql b/tests/queries/0_stateless/02967_prewhere_no_columns.sql new file mode 100644 index 00000000000..efcc952caa2 --- /dev/null +++ b/tests/queries/0_stateless/02967_prewhere_no_columns.sql @@ -0,0 +1,51 @@ +CREATE TABLE t_02967 +( + `key` Date, + `value` UInt16 +) +ENGINE = MergeTree +ORDER BY key +SETTINGS + index_granularity_bytes = 0 --8192 --, min_index_granularity_bytes = 2 + , index_granularity = 100 + , min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0 +-- +-- , min_bytes_for_wide_part = 2 +AS SELECT + number, + repeat(toString(number), 5) +FROM numbers(105.); + + + +-- Check with newly inserted data part. It's in-memory structured are filled at insert time. +SELECT + count(ignore(*)) +FROM t_02967 +PREWHERE CAST(ignore() + 1 as UInt8) +GROUP BY + ignore(65535, *), + ignore(255, 256, *) +SETTINGS + --send_logs_level='test', + max_threads=1; + + + +-- Reload part form disk to check that in-meory structures where properly serilaized-deserialized +DETACH TABLE t_02967; +ATTACH TABLE t_02967; + + +SELECT + count(ignore(*)) +FROM t_02967 +PREWHERE CAST(ignore() + 1 as UInt8) +GROUP BY + ignore(65535, *), + ignore(255, 256, *) +SETTINGS + --send_logs_level='test', + max_threads=1; + +DROP TABLE t_02967; From a11a5e783b0089a943f96ce58fedcf15bb586fc0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 24 Jan 2024 18:15:46 +0100 Subject: [PATCH 002/107] Adjust last granule after reading row count --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87f23b0da2a..d7221f5a536 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1176,6 +1176,14 @@ void IMergeTreeDataPart::loadRowsCount() auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); + + if (!index_granularity.empty() && rows_count < index_granularity.getTotalRows() && index_granularity_info.fixed_index_granularity) + { + /// Adjust last granule size to match the number of rows in the part in case of fixed index_granularity. + index_granularity.popMark(); + index_granularity.appendMark(rows_count % index_granularity_info.fixed_index_granularity); + chassert(rows_count == index_granularity.getTotalRows()); + } }; if (index_granularity.empty()) From 367a874edd25ffe4c9cfc1e2105e9fad899c68bd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 24 Jan 2024 18:16:15 +0100 Subject: [PATCH 003/107] Adjust last granule when creating part --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 9d373504473..a670807a997 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -760,7 +760,7 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_ /// We can adjust marks only if we computed granularity for blocks. /// Otherwise we cannot change granularity because it will differ from /// other columns - if (compute_granularity && settings.can_use_adaptive_granularity) +// if (compute_granularity && settings.can_use_adaptive_granularity) { if (getCurrentMark() != index_granularity.getMarksCount() - 1) throw Exception(ErrorCodes::LOGICAL_ERROR, From 502b8239a219bdace74e1de32e337c97024c2bee Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 25 Jan 2024 13:40:02 +0100 Subject: [PATCH 004/107] Allow last mark not to match fixed granularity value --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a670807a997..bb60e682f1b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -535,7 +535,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) { - throw Exception( + /// With fixed granularity we can have last mark with less rows than granularity + const bool is_last_mark = (mark_num + 1 == index_granularity.getMarksCount()); + if (!data_part->index_granularity_info.fixed_index_granularity || !is_last_mark) + throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", From ca2bfdb9aec54bb0bf3c983bd0c62fcf2c48c1dd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 25 Jan 2024 13:41:49 +0100 Subject: [PATCH 005/107] Update according to last mark smaller size --- tests/queries/1_stateful/00166_explain_estimate.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00166_explain_estimate.reference b/tests/queries/1_stateful/00166_explain_estimate.reference index 71ddd681581..85ecd0b9a71 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.reference +++ b/tests/queries/1_stateful/00166_explain_estimate.reference @@ -1,5 +1,5 @@ test hits 1 57344 7 -test hits 1 8839168 1079 -test hits 1 835584 102 +test hits 1 8832938 1079 +test hits 1 829354 102 test hits 1 8003584 977 test hits 2 581632 71 From 02b349822f10c1026ee1bf1b3b0d4f213864d929 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 29 Jan 2024 19:03:34 +0100 Subject: [PATCH 006/107] Fix for last mark equal to fixed granule size --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d7221f5a536..8ebe39a916e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1182,7 +1182,10 @@ void IMergeTreeDataPart::loadRowsCount() /// Adjust last granule size to match the number of rows in the part in case of fixed index_granularity. index_granularity.popMark(); index_granularity.appendMark(rows_count % index_granularity_info.fixed_index_granularity); - chassert(rows_count == index_granularity.getTotalRows()); + if (rows_count != index_granularity.getTotalRows()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Index granularity total rows in part {} does not match rows_count: {}, instead of {}", + name, index_granularity.getTotalRows(), rows_count); } }; From 1f85889f889bfa35c8c7bc27fade1762b20906dd Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 29 Mar 2024 18:53:03 -0400 Subject: [PATCH 007/107] FuzzQuery table function --- src/Client/ClientBase.h | 2 +- src/{Client => Common}/QueryFuzzer.cpp | 50 +++-- src/{Client => Common}/QueryFuzzer.h | 1 + src/Storages/StorageFuzzQuery.cpp | 174 ++++++++++++++++++ src/Storages/StorageFuzzQuery.h | 89 +++++++++ src/Storages/registerStorages.cpp | 2 + src/TableFunctions/TableFunctionFuzzQuery.cpp | 54 ++++++ src/TableFunctions/TableFunctionFuzzQuery.h | 42 +++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + .../03031_table_function_fuzzquery.reference | 11 ++ .../03031_table_function_fuzzquery.sql | 18 ++ 12 files changed, 426 insertions(+), 19 deletions(-) rename src/{Client => Common}/QueryFuzzer.cpp (97%) rename src/{Client => Common}/QueryFuzzer.h (99%) create mode 100644 src/Storages/StorageFuzzQuery.cpp create mode 100644 src/Storages/StorageFuzzQuery.h create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.cpp create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.h create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.reference create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.sql diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 9ec87ababfc..c0188253904 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -6,13 +6,13 @@ #include #include #include +#include #include #include #include #include #include #include -#include #include #include #include diff --git a/src/Client/QueryFuzzer.cpp b/src/Common/QueryFuzzer.cpp similarity index 97% rename from src/Client/QueryFuzzer.cpp rename to src/Common/QueryFuzzer.cpp index 7be01686258..137d545f82f 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Common/QueryFuzzer.cpp @@ -68,22 +68,21 @@ Field QueryFuzzer::getRandomField(int type) { case 0: { - return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) - / sizeof(*bad_int64_values))]; + return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)]; } case 1: { static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, - FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)]; } case 2: { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], - static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) + bad_int64_values[fuzz_rand() % std::size(bad_int64_values)], + static_cast(scales[fuzz_rand() % std::size(scales)]) ); } default: @@ -165,7 +164,8 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + if (debug_output) + std::cerr << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,12 +174,14 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_output) + std::cerr << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + if (debug_output) + std::cerr << "inserted (0)\n"; } } @@ -197,7 +199,9 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + + if (debug_output) + std::cerr << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -206,12 +210,16 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + + if (debug_output) + std::cerr << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + + if (debug_output) + std::cerr << "inserted (0)\n"; } } @@ -344,7 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - std::cerr << "No random column.\n"; + if (debug_output) + std::cerr << "No random column.\n"; } } @@ -378,7 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - std::cerr << "No random column.\n"; + if (debug_output) + std::cerr << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1360,11 +1370,15 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - std::cout << std::endl; - WriteBufferFromOStream ast_buf(std::cout, 4096); - formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.finalize(); - std::cout << std::endl << std::endl; + if (debug_output) + { + std::cout << std::endl; + + WriteBufferFromOStream ast_buf(std::cout, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.finalize(); + std::cout << std::endl << std::endl; + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Common/QueryFuzzer.h similarity index 99% rename from src/Client/QueryFuzzer.h rename to src/Common/QueryFuzzer.h index 6165e589cae..8a83934b620 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -38,6 +38,7 @@ struct ASTWindowDefinition; struct QueryFuzzer { pcg64 fuzz_rand{randomSeed()}; + bool debug_output = true; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp new file mode 100644 index 00000000000..c29986c7a7a --- /dev/null +++ b/src/Storages/StorageFuzzQuery.cpp @@ -0,0 +1,174 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +ColumnPtr FuzzQuerySource::createColumn() +{ + auto column = ColumnString::create(); + ColumnString::Chars & data_to = column->getChars(); + ColumnString::Offsets & offsets_to = column->getOffsets(); + + offsets_to.resize(block_size); + IColumn::Offset offset = 0; + + for (size_t row_num = 0; row_num < block_size; ++row_num) + { + ASTPtr new_query = query->clone(); + fuzzer.fuzzMain(new_query); + + WriteBufferFromOwnString out; + formatAST(*new_query, out, false); + auto data = out.str(); + size_t data_len = data.size(); + + IColumn::Offset next_offset = offset + data_len + 1; + data_to.resize(next_offset); + + std::copy(data.begin(), data.end(), &data_to[offset]); + + data_to[offset + data_len] = 0; + offsets_to[row_num] = next_offset; + + offset = next_offset; + } + + return column; +} + +StorageFuzzQuery::StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_) + : IStorage(table_id_), config(config_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); +} + +Pipe StorageFuzzQuery::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + storage_snapshot->check(column_names); + + Pipes pipes; + pipes.reserve(num_streams); + + const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); + Block block_header; + for (const auto & name : column_names) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + const char * begin = config.query.data(); + const char * end = begin + config.query.size(); + + ParserQuery parser(end, 0); + auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + for (UInt64 i = 0; i < num_streams; ++i) + pipes.emplace_back(std::make_shared(max_block_size, block_header, config, query)); + + return Pipe::unitePipes(std::move(pipes)); +} + +static constexpr std::array optional_configuration_keys = {"query_str", "random_seed"}; + +void StorageFuzzQuery::processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection) +{ + validateNamedCollection( + collection, + std::unordered_set(), + std::unordered_set(optional_configuration_keys.begin(), optional_configuration_keys.end())); + + if (collection.has("query")) + configuration.query = collection.get("query"); + + if (collection.has("random_seed")) + configuration.random_seed = collection.get("random_seed"); +} + +StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) +{ + StorageFuzzQuery::Configuration configuration{}; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + { + StorageFuzzQuery::processNamedCollectionResult(configuration, *named_collection); + } + else + { + // Supported signatures: + // + // FuzzQuery('query') + // FuzzQuery('query', 'random_seed') + if (engine_args.empty() || engine_args.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 2 arguments: query, random_seed"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); + configuration.query = std::move(first_arg); + + if (engine_args.size() == 2) + { + const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); + } + } + return configuration; +} + +void registerStorageFuzzQuery(StorageFactory & factory) +{ + factory.registerStorage( + "FuzzQuery", + [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + + if (engine_args.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage FuzzQuery must have arguments."); + + StorageFuzzQuery::Configuration configuration = StorageFuzzQuery::getConfiguration(engine_args, args.getLocalContext()); + + for (const auto& col : args.columns) + if (col.type->getTypeId() != TypeIndex::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'StorageFuzzQuery' supports only columns of String type, got {}.", col.type->getName()); + + return std::make_shared(args.table_id, args.columns, args.comment, configuration); + }); +} + +} diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h new file mode 100644 index 00000000000..47142a81f16 --- /dev/null +++ b/src/Storages/StorageFuzzQuery.h @@ -0,0 +1,89 @@ +#pragma once + +#include +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class NamedCollection; + +class StorageFuzzQuery final : public IStorage +{ +public: + struct Configuration : public StatelessTableEngineConfiguration + { + String query = ""; + UInt64 random_seed = randomSeed(); + }; + + StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_); + + std::string getName() const override { return "FuzzQuery"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection); + + static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); + +private: + const Configuration config; +}; + + +class FuzzQuerySource : public ISource +{ +public: + FuzzQuerySource( + UInt64 block_size_, Block block_header_, const StorageFuzzQuery::Configuration & config_, ASTPtr query_) + : ISource(block_header_) + , block_size(block_size_) + , block_header(std::move(block_header_)) + , config(config_) + , query(query_) + { + fuzzer.fuzz_rand = config_.random_seed; + } + + String getName() const override { return "FuzzQuery"; } + +protected: + Chunk generate() override + { + Columns columns; + columns.reserve(block_header.columns()); + for (const auto & col : block_header) + { + chassert(col.type->getTypeId() == TypeIndex::String); + columns.emplace_back(createColumn()); + } + + return {std::move(columns), block_size}; + } + +private: + ColumnPtr createColumn(); + + UInt64 block_size; + Block block_header; + + StorageFuzzQuery::Configuration config; + ASTPtr query; + + QueryFuzzer fuzzer; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index dea9feaf28b..f66d3ec3bfc 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -25,6 +25,7 @@ void registerStorageLiveView(StorageFactory & factory); void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); +void registerStorageFuzzQuery(StorageFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -126,6 +127,7 @@ void registerStorages() registerStorageGenerateRandom(factory); registerStorageExecutable(factory); registerStorageWindowView(factory); + registerStorageFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif diff --git a/src/TableFunctions/TableFunctionFuzzQuery.cpp b/src/TableFunctions/TableFunctionFuzzQuery.cpp new file mode 100644 index 00000000000..224f6666556 --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.cpp @@ -0,0 +1,54 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +void TableFunctionFuzzQuery::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); + + auto args = args_func.at(0)->children; + configuration = StorageFuzzQuery::getConfiguration(args, context); +} + +StoragePtr TableFunctionFuzzQuery::executeImpl( + const ASTPtr & /*ast_function*/, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool is_insert_query) const +{ + ColumnsDescription columns = getActualTableStructure(context, is_insert_query); + auto res = std::make_shared( + StorageID(getDatabaseName(), table_name), + columns, + /* comment */ String{}, + configuration); + res->startup(); + return res; +} + +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = "Perturbs a query string with random variations.", + .returned_value = "A table object with a single column containing perturbed query strings."}, + .allow_readonly = true}); +} + +} diff --git a/src/TableFunctions/TableFunctionFuzzQuery.h b/src/TableFunctions/TableFunctionFuzzQuery.h new file mode 100644 index 00000000000..22d10341c4d --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class TableFunctionFuzzQuery : public ITableFunction +{ +public: + static constexpr auto name = "fuzzQuery"; + std::string getName() const override { return name; } + + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + ColumnsDescription getActualTableStructure(ContextPtr /* context */, bool /* is_insert_query */) const override + { + return ColumnsDescription{{"query", std::make_shared()}}; + } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return "fuzzQuery"; } + + String source; + std::optional random_seed; + StorageFuzzQuery::Configuration configuration; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 927457ff9f6..2952d0b7b70 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -25,6 +25,7 @@ void registerTableFunctions() registerTableFunctionMongoDB(factory); registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); + registerTableFunctionFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerTableFunctionFuzzJSON(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 296af146faf..eef262490bf 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -22,6 +22,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMergeTreeIndex(TableFunctionFactory & factory); +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerTableFunctionFuzzJSON(TableFunctionFactory & factory); #endif diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference new file mode 100644 index 00000000000..d598037127f --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -0,0 +1,11 @@ +SELECT 1 +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC +SELECT\n item_id,\n *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], toNullable(\'Array(String)\')) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(materialize(toLowCardinality(3)))\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString(count(), (number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC NULLS FIRST,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n [toString((number % 2) * 2)],\n CAST([toString(number % 2)], toNullable(\'Array(LowCardinality(String))\')) AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE (number % 2) * toUInt128(2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC +SELECT\n toString((number % 2) * 2),\n *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql new file mode 100644 index 00000000000..5f5bb4b23e4 --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -0,0 +1,18 @@ + +SELECT * FROM fuzzQuery('SELECT 1', 8956) LIMIT 1; + +SELECT * FROM fuzzQuery('SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; +', 8956) LIMIT 10; From 77e28e29e9b2550204ea8d60647115587f36673e Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 30 Mar 2024 14:40:18 -0400 Subject: [PATCH 008/107] fix style --- src/Storages/StorageFuzzQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp index c29986c7a7a..56b6a4de2a6 100644 --- a/src/Storages/StorageFuzzQuery.cpp +++ b/src/Storages/StorageFuzzQuery.cpp @@ -19,8 +19,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; - extern const int INCORRECT_DATA; } ColumnPtr FuzzQuerySource::createColumn() From 6e611f7e81fe9977dd88f81cc4c24f3e1620ae8d Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 8 Apr 2024 16:18:29 +0200 Subject: [PATCH 009/107] fix review --- .../table-functions/fuzzQuery.md | 35 +++++++++ src/Common/QueryFuzzer.cpp | 40 +++++----- src/Common/QueryFuzzer.h | 37 ++++++--- src/Storages/StorageFuzzQuery.cpp | 75 +++++++++---------- src/Storages/StorageFuzzQuery.h | 4 +- .../03031_table_function_fuzzquery.reference | 18 ++--- 6 files changed, 124 insertions(+), 85 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/fuzzQuery.md diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md new file mode 100644 index 00000000000..ff8cfd1cd3b --- /dev/null +++ b/docs/en/sql-reference/table-functions/fuzzQuery.md @@ -0,0 +1,35 @@ +--- +slug: /en/sql-reference/table-functions/fuzzQuery +sidebar_position: 75 +sidebar_label: fuzzQuery +--- + +# fuzzQuery + +Perturbs the given query string with random variations. + +``` sql +fuzzQuery(query[, random_seed]) +``` + +**Arguments** + +- `query` (String) - The source query to perform the fuzzing on. +- `random_seed` (UInt64) - A random seed for producing stable results. + +**Returned Value** + +A table object with a single column containing perturbed query strings. + +## Usage Example + +``` sql +SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2; +``` + +``` + ┌─query──────────────────────────────────────────────────────────┐ +1. │ SELECT 'a' AS key GROUP BY key │ +2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │ + └────────────────────────────────────────────────────────────────┘ +``` diff --git a/src/Common/QueryFuzzer.cpp b/src/Common/QueryFuzzer.cpp index 137d545f82f..6dd51033e3c 100644 --- a/src/Common/QueryFuzzer.cpp +++ b/src/Common/QueryFuzzer.cpp @@ -164,8 +164,8 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - if (debug_output) - std::cerr << "erased\n"; + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,14 +174,14 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - if (debug_output) - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - if (debug_output) - std::cerr << "inserted (0)\n"; + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -200,8 +200,8 @@ Field QueryFuzzer::fuzzField(Field field) size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - if (debug_output) - std::cerr << "erased\n"; + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -211,15 +211,15 @@ Field QueryFuzzer::fuzzField(Field field) size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - if (debug_output) - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - if (debug_output) - std::cerr << "inserted (0)\n"; + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -352,8 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - if (debug_output) - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } } @@ -387,8 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - if (debug_output) - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1370,14 +1370,14 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - if (debug_output) + if (out_stream) { - std::cout << std::endl; + *out_stream << std::endl; - WriteBufferFromOStream ast_buf(std::cout, 4096); + WriteBufferFromOStream ast_buf(*out_stream, 4096); formatAST(*ast, ast_buf, false /*highlight*/); ast_buf.finalize(); - std::cout << std::endl << std::endl; + *out_stream << std::endl << std::endl; } } diff --git a/src/Common/QueryFuzzer.h b/src/Common/QueryFuzzer.h index 8a83934b620..3cf0381e044 100644 --- a/src/Common/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -35,10 +35,32 @@ struct ASTWindowDefinition; * queries, so you want to feed it a lot of queries to get some interesting mix * of them. Normally we feed SQL regression tests to it. */ -struct QueryFuzzer +class QueryFuzzer { - pcg64 fuzz_rand{randomSeed()}; - bool debug_output = true; +public: + + QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = &std::cout, std::ostream * debug_stream_ = &std::cerr) + : fuzz_rand(fuzz_rand_) + , out_stream(out_stream_) + , debug_stream(debug_stream_) + { + } + + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + + ASTs getInsertQueriesForFuzzedTables(const String & full_query); + ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); + void notifyQueryFailed(ASTPtr ast); + + static bool isSuitableForFuzzing(const ASTCreateQuery & create); + +private: + pcg64 fuzz_rand; + + std::ostream * out_stream = nullptr; + std::ostream * debug_stream = nullptr; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is @@ -67,10 +89,6 @@ struct QueryFuzzer std::unordered_map index_of_fuzzed_table; std::set created_tables_hashes; - // This is the only function you have to call -- it will modify the passed - // ASTPtr to point to new AST with some random changes. - void fuzzMain(ASTPtr & ast); - // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); @@ -78,9 +96,6 @@ struct QueryFuzzer ASTPtr getRandomExpressionList(); DataTypePtr fuzzDataType(DataTypePtr type); DataTypePtr getRandomType(); - ASTs getInsertQueriesForFuzzedTables(const String & full_query); - ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); - void notifyQueryFailed(ASTPtr ast); void replaceWithColumnLike(ASTPtr & ast); void replaceWithTableLike(ASTPtr & ast); void fuzzOrderByElement(ASTOrderByElement * elem); @@ -103,8 +118,6 @@ struct QueryFuzzer void addTableLike(ASTPtr ast); void addColumnLike(ASTPtr ast); void collectFuzzInfoRecurse(ASTPtr ast); - - static bool isSuitableForFuzzing(const ASTCreateQuery & create); }; } diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp index 56b6a4de2a6..e2b836c98b9 100644 --- a/src/Storages/StorageFuzzQuery.cpp +++ b/src/Storages/StorageFuzzQuery.cpp @@ -30,16 +30,29 @@ ColumnPtr FuzzQuerySource::createColumn() offsets_to.resize(block_size); IColumn::Offset offset = 0; - for (size_t row_num = 0; row_num < block_size; ++row_num) + auto fuzz_base = query; + size_t row_num = 0; + + while (row_num < block_size) { - ASTPtr new_query = query->clone(); + ASTPtr new_query = fuzz_base->clone(); + + auto base_before_fuzz = fuzz_base->formatForErrorMessage(); fuzzer.fuzzMain(new_query); + auto fuzzed_text = new_query->formatForErrorMessage(); WriteBufferFromOwnString out; formatAST(*new_query, out, false); auto data = out.str(); size_t data_len = data.size(); + /// AST is too long, will start from the original query. + if (data_len > 500) + { + fuzz_base = query; + continue; + } + IColumn::Offset next_offset = offset + data_len + 1; data_to.resize(next_offset); @@ -49,6 +62,8 @@ ColumnPtr FuzzQuerySource::createColumn() offsets_to[row_num] = next_offset; offset = next_offset; + fuzz_base = new_query; + ++row_num; } return column; @@ -99,52 +114,30 @@ Pipe StorageFuzzQuery::read( return Pipe::unitePipes(std::move(pipes)); } -static constexpr std::array optional_configuration_keys = {"query_str", "random_seed"}; - -void StorageFuzzQuery::processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection) -{ - validateNamedCollection( - collection, - std::unordered_set(), - std::unordered_set(optional_configuration_keys.begin(), optional_configuration_keys.end())); - - if (collection.has("query")) - configuration.query = collection.get("query"); - - if (collection.has("random_seed")) - configuration.random_seed = collection.get("random_seed"); -} - StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) { StorageFuzzQuery::Configuration configuration{}; - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + // Supported signatures: + // + // FuzzQuery('query') + // FuzzQuery('query', 'random_seed') + if (engine_args.empty() || engine_args.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 2 arguments: query, random_seed"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); + configuration.query = std::move(first_arg); + + if (engine_args.size() == 2) { - StorageFuzzQuery::processNamedCollectionResult(configuration, *named_collection); + const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); } - else - { - // Supported signatures: - // - // FuzzQuery('query') - // FuzzQuery('query', 'random_seed') - if (engine_args.empty() || engine_args.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 2 arguments: query, random_seed"); - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); - configuration.query = std::move(first_arg); - - if (engine_args.size() == 2) - { - const auto & literal = engine_args[1]->as(); - if (!literal.value.isNull()) - configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); - } - } return configuration; } diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h index 47142a81f16..e948d0b2acf 100644 --- a/src/Storages/StorageFuzzQuery.h +++ b/src/Storages/StorageFuzzQuery.h @@ -35,8 +35,6 @@ public: size_t max_block_size, size_t num_streams) override; - static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection); - static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); private: @@ -54,8 +52,8 @@ public: , block_header(std::move(block_header_)) , config(config_) , query(query_) + , fuzzer(config_.random_seed, /* out_stream= */ nullptr, /* debug_stream= */ nullptr) { - fuzzer.fuzz_rand = config_.random_seed; } String getName() const override { return "FuzzQuery"; } diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference index d598037127f..c5b92291207 100644 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -1,11 +1,11 @@ SELECT 1 SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC -SELECT\n item_id,\n *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], toNullable(\'Array(String)\')) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(materialize(toLowCardinality(3)))\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString(count(), (number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC NULLS FIRST,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n [toString((number % 2) * 2)],\n CAST([toString(number % 2)], toNullable(\'Array(LowCardinality(String))\')) AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE (number % 2) * toUInt128(2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC -SELECT\n toString((number % 2) * 2),\n *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(materialize(toLowCardinality(3)))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC +SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC NULLS LAST +SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\'),\n CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC NULLS FIRST +SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\'),\n CAST([toString(multiply(number % 2, item_id, 2))]) AS item_id\n FROM numbers(3)\n WHERE \'Array(LowCardinality(String))\'\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 DESC NULLS FIRST +SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC From 4e1f98ee7b78d48362a6788109658c8de859abd5 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 9 Apr 2024 14:53:28 +0200 Subject: [PATCH 010/107] removed cout,cerr from src --- src/Client/ClientBase.h | 2 +- src/Common/QueryFuzzer.h | 3 +-- src/Storages/StorageFuzzQuery.h | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index c0188253904..2b05878c176 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -298,7 +298,7 @@ protected: bool send_external_tables = false; NameToNameMap query_parameters; /// Dictionary with query parameters for prepared statements. - QueryFuzzer fuzzer; + QueryFuzzer fuzzer{randomSeed(), &std::cout, &std::cerr}; int query_fuzzer_runs = 0; int create_query_fuzzer_runs = 0; diff --git a/src/Common/QueryFuzzer.h b/src/Common/QueryFuzzer.h index 3cf0381e044..bf87bdfb24e 100644 --- a/src/Common/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -38,8 +38,7 @@ struct ASTWindowDefinition; class QueryFuzzer { public: - - QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = &std::cout, std::ostream * debug_stream_ = &std::cerr) + QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) : fuzz_rand(fuzz_rand_) , out_stream(out_stream_) , debug_stream(debug_stream_) diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h index e948d0b2acf..40833190895 100644 --- a/src/Storages/StorageFuzzQuery.h +++ b/src/Storages/StorageFuzzQuery.h @@ -52,7 +52,7 @@ public: , block_header(std::move(block_header_)) , config(config_) , query(query_) - , fuzzer(config_.random_seed, /* out_stream= */ nullptr, /* debug_stream= */ nullptr) + , fuzzer(config_.random_seed) { } From 3bbf86d34506b74a92ca53e857d4c35b9406a08b Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 9 Apr 2024 14:54:58 +0200 Subject: [PATCH 011/107] update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ee3ef1ae795..9d9bea11b82 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1581,6 +1581,7 @@ fuzzBits fuzzJSON fuzzer fuzzers +fuzzQuery gRPC gccMurmurHash gcem From 6329dc812462363ff6edf0388239f6bead2efacd Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 9 Apr 2024 15:05:49 +0200 Subject: [PATCH 012/107] fix --- programs/client/Client.h | 5 ++++- src/Client/ClientBase.h | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.h b/programs/client/Client.h index 11d9dec97b1..122b8e5ab3f 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -9,7 +9,10 @@ namespace DB class Client : public ClientBase { public: - Client() = default; + Client() + { + fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr); + } void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 2b05878c176..c0188253904 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -298,7 +298,7 @@ protected: bool send_external_tables = false; NameToNameMap query_parameters; /// Dictionary with query parameters for prepared statements. - QueryFuzzer fuzzer{randomSeed(), &std::cout, &std::cerr}; + QueryFuzzer fuzzer; int query_fuzzer_runs = 0; int create_query_fuzzer_runs = 0; From 76415ba3523921de138feb39fca7fbc65b8f6dc5 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 10 Apr 2024 15:27:53 +0200 Subject: [PATCH 013/107] add `explicit`, more stable tests --- src/Common/QueryFuzzer.h | 2 +- .../03031_table_function_fuzzquery.reference | 13 ++----------- .../0_stateless/03031_table_function_fuzzquery.sql | 4 ++-- 3 files changed, 5 insertions(+), 14 deletions(-) diff --git a/src/Common/QueryFuzzer.h b/src/Common/QueryFuzzer.h index bf87bdfb24e..35d088809f2 100644 --- a/src/Common/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -38,7 +38,7 @@ struct ASTWindowDefinition; class QueryFuzzer { public: - QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) + explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) : fuzz_rand(fuzz_rand_) , out_stream(out_stream_) , debug_stream(debug_stream_) diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference index c5b92291207..202e4557a33 100644 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -1,11 +1,2 @@ -SELECT 1 -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(materialize(toLowCardinality(3)))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC -SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 ASC NULLS LAST -SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(toLowCardinality(3))\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\'),\n CAST([toString((number % 2) * 2)]) AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 ASC,\n 3 DESC NULLS FIRST -SELECT *\nFROM\n(\n SELECT CAST(\'Array(LowCardinality(String))\') AS item_id\n FROM numbers(3)\n WHERE toString(number % 2)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\'),\n CAST([toString(multiply(number % 2, item_id, 2))]) AS item_id\n FROM numbers(3)\n WHERE \'Array(LowCardinality(String))\'\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 DESC NULLS FIRST -SELECT *\nFROM\n(\n SELECT\n CAST([toString(number % 2)], \'Array(LowCardinality(String))\') AS item_id,\n count()\n FROM numbers(3)\n GROUP BY item_id\n WITH TOTALS\n) AS l\nFULL OUTER JOIN\n(\n SELECT CAST([toString((number % 2) * 2)], \'Array(String)\') AS item_id\n FROM numbers(3)\n) AS r ON l.item_id = r.item_id\nORDER BY\n 1 ASC,\n 2 DESC,\n 3 ASC +query +String diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql index 5f5bb4b23e4..5821e2e5111 100644 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -1,5 +1,5 @@ -SELECT * FROM fuzzQuery('SELECT 1', 8956) LIMIT 1; +SELECT * FROM fuzzQuery('SELECT 1', 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; SELECT * FROM fuzzQuery('SELECT * FROM ( @@ -15,4 +15,4 @@ FROM ( ) AS r ON l.item_id = r.item_id ORDER BY 1,2,3; -', 8956) LIMIT 10; +', 8956) LIMIT 10 FORMAT NULL; From 4df3cf3b151f407c4daaa7f8df80a6fddb836280 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 11 Apr 2024 16:57:45 +0200 Subject: [PATCH 014/107] Update StorageFuzzQuery.h --- src/Storages/StorageFuzzQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h index 40833190895..3ae506fdfb8 100644 --- a/src/Storages/StorageFuzzQuery.h +++ b/src/Storages/StorageFuzzQuery.h @@ -17,7 +17,7 @@ class StorageFuzzQuery final : public IStorage public: struct Configuration : public StatelessTableEngineConfiguration { - String query = ""; + String query; UInt64 random_seed = randomSeed(); }; From 562d76ccab6ffc53c9059139fead3b2e41ff4725 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 12 Apr 2024 11:19:44 +0200 Subject: [PATCH 015/107] Update StorageFuzzQuery.cpp --- src/Storages/StorageFuzzQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp index e2b836c98b9..5e29a04427b 100644 --- a/src/Storages/StorageFuzzQuery.cpp +++ b/src/Storages/StorageFuzzQuery.cpp @@ -105,7 +105,7 @@ Pipe StorageFuzzQuery::read( const char * begin = config.query.data(); const char * end = begin + config.query.size(); - ParserQuery parser(end, 0); + ParserQuery parser(end, false); auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); for (UInt64 i = 0; i < num_streams; ++i) From 9ed3acce8223485798b22bcffcd7bd8d595cd025 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 10 May 2024 18:22:23 +0000 Subject: [PATCH 016/107] refactoring near azure blob storage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 96 +++--- src/Backups/BackupIO_AzureBlobStorage.h | 15 +- .../registerBackupEngineAzureBlobStorage.cpp | 66 ++-- src/Core/Settings.h | 31 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 4 +- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- .../AzureBlobStorage/AzureBlobStorageAuth.h | 58 ---- ...ageAuth.cpp => AzureBlobStorageCommon.cpp} | 276 +++++++++------- .../AzureBlobStorage/AzureBlobStorageCommon.h | 139 ++++++++ .../AzureBlobStorage/AzureObjectStorage.cpp | 37 ++- .../AzureBlobStorage/AzureObjectStorage.h | 75 +---- .../Cached/CachedObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 2 +- .../ObjectStorages/ObjectStorageFactory.cpp | 17 +- .../copyAzureBlobStorageFile.cpp | 9 +- .../copyAzureBlobStorageFile.h | 4 +- src/Storages/StorageAzureBlob.cpp | 308 +++++------------- src/Storages/StorageAzureBlob.h | 26 +- src/Storages/StorageAzureBlobCluster.cpp | 5 +- .../TableFunctionAzureBlobStorage.cpp | 65 ++-- .../TableFunctionAzureBlobStorageCluster.cpp | 17 +- 21 files changed, 604 insertions(+), 650 deletions(-) delete mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h rename src/Disks/ObjectStorages/AzureBlobStorage/{AzureBlobStorageAuth.cpp => AzureBlobStorageCommon.cpp} (53%) create mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index a3998431674..3f60ed5c0b4 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -30,21 +31,21 @@ namespace ErrorCodes } BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const AzureBlobStorage::ConnectionParams & connection_params_, + const String & blob_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} - , configuration(configuration_) + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.endpoint.container_name, false, false} + , connection_params(connection_params_) + , blob_path(blob_path_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); - client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); - + auto client_ptr = AzureBlobStorage::getContainerClient(connection_params, /*readonly=*/ false); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration_.container); + AzureBlobStorage::getRequestSettings(context_->getSettingsRef()), + connection_params.endpoint.container_name); client = object_storage->getAzureBlobStorageClient(); settings = object_storage->getSettings(); } @@ -53,20 +54,20 @@ BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; bool BackupReaderAzureBlobStorage::fileExists(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; return object_storage->exists(StoredObject(key)); } UInt64 BackupReaderAzureBlobStorage::getFileSize(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); return object_metadata.size_bytes; } std::unique_ptr BackupReaderAzureBlobStorage::readFile(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; return std::make_unique( client, key, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); @@ -81,23 +82,23 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); - auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t + auto write_blob_function = [&](const Strings & dst_blob_path, WriteMode mode, const std::optional &) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. - if (blob_path.size() != 2 || mode != WriteMode::Rewrite) + if (dst_blob_path.size() != 2 || mode != WriteMode::Rewrite) throw Exception(ErrorCodes::LOGICAL_ERROR, "Blob writing function called with unexpected blob_path.size={} or mode={}", - blob_path.size(), mode); + dst_blob_path.size(), mode); copyAzureBlobStorageFile( client, destination_disk->getObjectStorage()->getAzureBlobStorageClient(), - configuration.container, - fs::path(configuration.blob_path) / path_in_backup, + connection_params.endpoint.container_name, + fs::path(blob_path) / path_in_backup, 0, file_size, - /* dest_container */ blob_path[1], - /* dest_path */ blob_path[0], + /* dest_container */ dst_blob_path[1], + /* dest_path */ dst_blob_path[0], settings, read_settings, threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupRDAzure")); @@ -115,22 +116,25 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const AzureBlobStorage::ConnectionParams & connection_params_, + const String & blob_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} - , configuration(configuration_) + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.endpoint.container_name, false, false} + , connection_params(connection_params_) + , blob_path(blob_path_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); - client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); + if (!attempt_to_create_container) + connection_params.endpoint.container_already_exists = true; + auto client_ptr = AzureBlobStorage::getContainerClient(connection_params, /*readonly=*/ false); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration_.container); + AzureBlobStorage::getRequestSettings(context_->getSettingsRef()), + connection_params.endpoint.container_name); client = object_storage->getAzureBlobStorageClient(); settings = object_storage->getSettings(); } @@ -144,18 +148,18 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage container. /// In this case we can't use the native copy. - if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) + if (auto src_blob_path = src_disk->getBlobPath(src_path); src_blob_path.size() == 2) { LOG_TRACE(log, "Copying file {} from disk {} to AzureBlobStorag", src_path, src_disk->getName()); copyAzureBlobStorageFile( src_disk->getObjectStorage()->getAzureBlobStorageClient(), client, - /* src_container */ blob_path[1], - /* src_path */ blob_path[0], + /* src_container */ src_blob_path[1], + /* src_path */ src_blob_path[0], start_pos, length, - configuration.container, - fs::path(configuration.blob_path) / path_in_backup, + connection_params.endpoint.container_name, + fs::path(blob_path) / path_in_backup, settings, read_settings, threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); @@ -173,11 +177,11 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St copyAzureBlobStorageFile( client, client, - configuration.container, - fs::path(configuration.blob_path)/ source, + connection_params.endpoint.container_name, + fs::path(blob_path)/ source, 0, size, - /* dest_container */ configuration.container, + /* dest_container */ connection_params.endpoint.container_name, /* dest_path */ destination, settings, read_settings, @@ -186,21 +190,29 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, fs::path(configuration.blob_path) / path_in_backup, settings, - threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); + copyDataToAzureBlobStorageFile( + create_read_buffer, + start_pos, + length, + client, + connection_params.endpoint.container_name, + fs::path(blob_path) / path_in_backup, + settings, + threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), + "BackupWRAzure")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; bool BackupWriterAzureBlobStorage::fileExists(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; return object_storage->exists(StoredObject(key)); } UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; RelativePathsWithMetadata children; object_storage->listObjects(key,children,/*max_keys*/0); if (children.empty()) @@ -210,7 +222,7 @@ UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; return std::make_unique( client, key, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); @@ -218,7 +230,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; return std::make_unique( client, key, @@ -230,7 +242,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin void BackupWriterAzureBlobStorage::removeFile(const String & file_name) { - String key = fs::path(configuration.blob_path) / file_name; + String key = fs::path(blob_path) / file_name; StoredObject object(key); object_storage->removeObjectIfExists(object); } @@ -239,7 +251,7 @@ void BackupWriterAzureBlobStorage::removeFiles(const Strings & file_names) { StoredObjects objects; for (const auto & file_name : file_names) - objects.emplace_back(fs::path(configuration.blob_path) / file_name); + objects.emplace_back(fs::path(blob_path) / file_name); object_storage->removeObjectsIfExist(objects); @@ -249,7 +261,7 @@ void BackupWriterAzureBlobStorage::removeFilesBatch(const Strings & file_names) { StoredObjects objects; for (const auto & file_name : file_names) - objects.emplace_back(fs::path(configuration.blob_path) / file_name); + objects.emplace_back(fs::path(blob_path) / file_name); object_storage->removeObjectsIfExist(objects); } diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index f0b9aace4d4..0829c3258c9 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -1,5 +1,6 @@ #pragma once +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -16,7 +17,7 @@ namespace DB class BackupReaderAzureBlobStorage : public BackupReaderDefault { public: - BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + BackupReaderAzureBlobStorage(const AzureBlobStorage::ConnectionParams & connection_params_, const String & blob_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); ~BackupReaderAzureBlobStorage() override; bool fileExists(const String & file_name) override; @@ -29,15 +30,16 @@ public: private: const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + AzureBlobStorage::ConnectionParams connection_params; + String blob_path; std::unique_ptr object_storage; - std::shared_ptr settings; + std::shared_ptr settings; }; class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: - BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container); + BackupWriterAzureBlobStorage(const AzureBlobStorage::ConnectionParams & connection_params_, const String & blob_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container); ~BackupWriterAzureBlobStorage() override; bool fileExists(const String & file_name) override; @@ -58,9 +60,10 @@ private: void removeFilesBatch(const Strings & file_names); const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + AzureBlobStorage::ConnectionParams connection_params; + String blob_path; std::unique_ptr object_storage; - std::shared_ptr settings; + std::shared_ptr settings; }; } diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 1b9545fc455..6974d16e2f6 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -1,3 +1,4 @@ +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #include @@ -49,7 +50,9 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; - StorageAzureBlob::Configuration configuration; + String blob_path; + AzureBlobStorage::ConnectionParams connection_params; + auto request_settings = AzureBlobStorage::getRequestSettings(params.context->getSettingsRef()); if (!id_arg.empty()) { @@ -59,54 +62,41 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (!config.has(config_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", id_arg); - if (config.has(config_prefix + ".connection_string")) + connection_params = { - configuration.connection_url = config.getString(config_prefix + ".connection_string"); - configuration.is_connection_string = true; - configuration.container = config.getString(config_prefix + ".container"); - } - else - { - configuration.connection_url = config.getString(config_prefix + ".storage_account_url"); - configuration.is_connection_string = false; - configuration.container = config.getString(config_prefix + ".container"); - configuration.account_name = config.getString(config_prefix + ".account_name"); - configuration.account_key = config.getString(config_prefix + ".account_key"); - - if (config.has(config_prefix + ".account_name") && config.has(config_prefix + ".account_key")) - { - configuration.account_name = config.getString(config_prefix + ".account_name"); - configuration.account_key = config.getString(config_prefix + ".account_key"); - } - } + .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), + .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), + .client_options = AzureBlobStorage::getClientOptions(*request_settings, /*for_disk=*/ true), + }; if (args.size() > 1) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); if (args.size() == 1) - configuration.blob_path = args[0].safeGet(); - + blob_path = args[0].safeGet(); } else { if (args.size() == 3) { - configuration.connection_url = args[0].safeGet(); - configuration.is_connection_string = !configuration.connection_url.starts_with("http"); + auto connection_url = args[0].safeGet(); + auto container_name = args[1].safeGet(); + blob_path = args[2].safeGet(); - configuration.container = args[1].safeGet(); - configuration.blob_path = args[2].safeGet(); + AzureBlobStorage::processURL(connection_url, container_name, connection_params.endpoint, connection_params.auth_method); + connection_params.client_options = AzureBlobStorage::getClientOptions(*request_settings, /*for_disk=*/ true); } else if (args.size() == 5) { - configuration.connection_url = args[0].safeGet(); - configuration.is_connection_string = false; + connection_params.endpoint.storage_account_url = args[0].safeGet(); + connection_params.endpoint.container_name = args[1].safeGet(); + blob_path = args[2].safeGet(); - configuration.container = args[1].safeGet(); - configuration.blob_path = args[2].safeGet(); - configuration.account_name = args[3].safeGet(); - configuration.account_key = args[4].safeGet(); + auto account_name = args[3].safeGet(); + auto account_key = args[4].safeGet(); + connection_params.auth_method = std::make_shared(account_name, account_key); + connection_params.client_options = AzureBlobStorage::getClientOptions(*request_settings, /*for_disk=*/ true); } else { @@ -116,12 +106,12 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } BackupImpl::ArchiveParams archive_params; - if (hasRegisteredArchiveFileExtension(configuration.blob_path)) + if (hasRegisteredArchiveFileExtension(blob_path)) { if (params.is_internal_backup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled"); - archive_params.archive_name = removeFileNameFromURL(configuration.blob_path); + archive_params.archive_name = removeFileNameFromURL(blob_path); archive_params.compression_method = params.compression_method; archive_params.compression_level = params.compression_level; archive_params.password = params.password; @@ -135,7 +125,9 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(configuration, + auto reader = std::make_shared( + connection_params, + blob_path, params.read_settings, params.write_settings, params.context); @@ -150,7 +142,9 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } else { - auto writer = std::make_shared(configuration, + auto writer = std::make_shared( + connection_params, + blob_path, params.read_settings, params.write_settings, params.context, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 04029983d84..e09fef794d7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -79,23 +79,13 @@ class IColumn; M(UInt64, distributed_connections_pool_size, 1024, "Maximum number of connections with one remote server in the pool.", 0) \ M(UInt64, connections_with_failover_max_tries, 3, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \ - M(UInt64, azure_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ - M(UInt64, azure_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage.", 0) \ - M(UInt64, azure_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ - M(UInt64, azure_upload_part_size_multiply_factor, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage.", 0) \ - M(UInt64, azure_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor.", 0) \ M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ - M(UInt64, azure_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ - M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ - M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ - M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ - M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ @@ -105,20 +95,33 @@ class IColumn; M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_use_adaptive_timeouts, true, "When adaptive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ - M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ - M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ - M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ - M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ 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_connect_timeout_ms, 1000, "Connection timeout for host from s3 disks.", 0) \ + M(UInt64, azure_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to Azure blob storage.", 0) \ + M(UInt64, azure_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage.", 0) \ + M(UInt64, azure_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage.", 0) \ + M(UInt64, azure_upload_part_size_multiply_factor, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage.", 0) \ + M(UInt64, azure_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor.", 0) \ + M(UInt64, azure_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ + M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ + M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ + M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ + M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ + M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ + M(UInt64, azure_sdk_max_retries, 10, "Maximum number of retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff beetween retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff beetween retries in azure sdk", 0) \ + M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ + M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ + M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 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/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..cadae33e23e 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -26,7 +26,7 @@ struct WriteBufferFromAzureBlobStorage::PartData std::string block_id; }; -BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) +BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureBlobStorage::RequestSettings & settings) { BufferAllocationPolicy::Settings allocation_settings; allocation_settings.strict_size = settings.strict_upload_part_size; @@ -44,7 +44,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( const String & blob_path_, size_t buf_size_, const WriteSettings & write_settings_, - std::shared_ptr settings_, + std::shared_ptr settings_, ThreadPoolCallbackRunnerUnsafe schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 96ba6acefff..f47ba92beab 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -35,7 +35,7 @@ public: const String & blob_path_, size_t buf_size_, const WriteSettings & write_settings_, - std::shared_ptr settings_, + std::shared_ptr settings_, ThreadPoolCallbackRunnerUnsafe schedule_ = {}); ~WriteBufferFromAzureBlobStorage() override; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h deleted file mode 100644 index e4775a053c1..00000000000 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ /dev/null @@ -1,58 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include - -namespace DB -{ - -struct AzureBlobStorageEndpoint -{ - const String storage_account_url; - const String account_name; - const String container_name; - const String prefix; - const std::optional container_already_exists; - - String getEndpoint() - { - String url = storage_account_url; - if (url.ends_with('/')) - url.pop_back(); - - if (!account_name.empty()) - url += "/" + account_name; - - if (!container_name.empty()) - url += "/" + container_name; - - if (!prefix.empty()) - url += "/" + prefix; - - return url; - } - - String getEndpointWithoutContainer() - { - String url = storage_account_url; - - if (!account_name.empty()) - url += "/" + account_name; - - return url; - } -}; - -std::unique_ptr getAzureBlobContainerClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); - -AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); - -std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); - -} - -#endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp similarity index 53% rename from src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp rename to src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index a535b007541..76054efff19 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -1,4 +1,8 @@ -#include +#include +#include +#include +#include +#include #if USE_AZURE_BLOB_STORAGE @@ -7,13 +11,9 @@ #include #include #include -#include #include #include -using namespace Azure::Storage::Blobs; - - namespace DB { @@ -22,8 +22,10 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace AzureBlobStorage +{ -void validateStorageAccountUrl(const String & storage_account_url) +static void validateStorageAccountUrl(const String & storage_account_url) { const auto * storage_account_url_pattern_str = R"(http(()|s)://[a-z0-9-.:]+(()|/)[a-z0-9]*(()|/))"; static const RE2 storage_account_url_pattern(storage_account_url_pattern_str); @@ -33,8 +35,7 @@ void validateStorageAccountUrl(const String & storage_account_url) "Blob Storage URL is not valid, should follow the format: {}, got: {}", storage_account_url_pattern_str, storage_account_url); } - -void validateContainerName(const String & container_name) +static void validateContainerName(const String & container_name) { auto len = container_name.length(); if (len < 3 || len > 64) @@ -50,13 +51,51 @@ void validateContainerName(const String & container_name) container_name_pattern_str, container_name); } +static bool isConnectionString(const std::string & candidate) +{ + return !candidate.starts_with("http"); +} -AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +String ConnectionParams::getConnectionURL() const +{ + if (std::holds_alternative(auth_method)) + { + auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(endpoint.storage_account_url); + return parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl(); + } + + return endpoint.storage_account_url; +} + +std::unique_ptr ConnectionParams::createForService() const +{ + return std::visit([this](const T & auth) + { + if constexpr (std::is_same_v) + return std::make_unique(ServiceClient::CreateFromConnectionString(auth.toUnderType(), client_options)); + else + return std::make_unique(endpoint.getEndpointWithoutContainer(), auth, client_options); + }, auth_method); +} + +std::unique_ptr ConnectionParams::createForContainer() const +{ + return std::visit([this](const T & auth) + { + if constexpr (std::is_same_v) + return std::make_unique(ContainerClient::CreateFromConnectionString(auth.toUnderType(), endpoint.container_name, client_options)); + else + return std::make_unique(endpoint.getEndpoint(), auth, client_options); + }, auth_method); +} + +Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { String storage_url; String account_name; String container_name; String prefix; + if (config.has(config_prefix + ".endpoint")) { String endpoint = config.getString(config_prefix + ".endpoint"); @@ -71,48 +110,48 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr if (endpoint_contains_account_name) { - size_t acc_pos_begin = endpoint.find('/', pos+2); + size_t acc_pos_begin = endpoint.find('/', pos + 2); if (acc_pos_begin == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected account_name in endpoint"); - storage_url = endpoint.substr(0,acc_pos_begin); - size_t acc_pos_end = endpoint.find('/',acc_pos_begin+1); + storage_url = endpoint.substr(0, acc_pos_begin); + size_t acc_pos_end = endpoint.find('/', acc_pos_begin + 1); if (acc_pos_end == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected container_name in endpoint"); - account_name = endpoint.substr(acc_pos_begin+1,(acc_pos_end-acc_pos_begin)-1); + account_name = endpoint.substr(acc_pos_begin + 1, acc_pos_end - acc_pos_begin - 1); - size_t cont_pos_end = endpoint.find('/', acc_pos_end+1); + size_t cont_pos_end = endpoint.find('/', acc_pos_end + 1); if (cont_pos_end != std::string::npos) { - container_name = endpoint.substr(acc_pos_end+1,(cont_pos_end-acc_pos_end)-1); - prefix = endpoint.substr(cont_pos_end+1); + container_name = endpoint.substr(acc_pos_end + 1, cont_pos_end - acc_pos_end - 1); + prefix = endpoint.substr(cont_pos_end + 1); } else { - container_name = endpoint.substr(acc_pos_end+1); + container_name = endpoint.substr(acc_pos_end + 1); } } else { - size_t cont_pos_begin = endpoint.find('/', pos+2); + size_t cont_pos_begin = endpoint.find('/', pos + 2); if (cont_pos_begin == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected container_name in endpoint"); - storage_url = endpoint.substr(0,cont_pos_begin); - size_t cont_pos_end = endpoint.find('/',cont_pos_begin+1); + storage_url = endpoint.substr(0, cont_pos_begin); + size_t cont_pos_end = endpoint.find('/', cont_pos_begin + 1); if (cont_pos_end != std::string::npos) { - container_name = endpoint.substr(cont_pos_begin+1,(cont_pos_end-cont_pos_begin)-1); - prefix = endpoint.substr(cont_pos_end+1); + container_name = endpoint.substr(cont_pos_begin + 1,cont_pos_end - cont_pos_begin - 1); + prefix = endpoint.substr(cont_pos_end + 1); } else { - container_name = endpoint.substr(cont_pos_begin+1); + container_name = endpoint.substr(cont_pos_begin + 1); } } } @@ -132,122 +171,117 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr if (!container_name.empty()) validateContainerName(container_name); + std::optional container_already_exists {}; if (config.has(config_prefix + ".container_already_exists")) container_already_exists = {config.getBool(config_prefix + ".container_already_exists")}; - return {storage_url, account_name, container_name, prefix, container_already_exists}; + + return {storage_url, account_name, container_name, prefix, "", container_already_exists}; } - -template -std::unique_ptr getClientWithConnectionString(const String & connection_str, const String & container_name, const BlobClientOptions & client_options) = delete; - -template<> -std::unique_ptr getClientWithConnectionString(const String & connection_str, const String & /*container_name*/, const BlobClientOptions & client_options) +void processURL(const String & url, const String & container_name, Endpoint & endpoint, AuthMethod & auth_method) { - return std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_str, client_options)); -} + endpoint.container_name = container_name; -template<> -std::unique_ptr getClientWithConnectionString(const String & connection_str, const String & container_name, const BlobClientOptions & client_options) -{ - return std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_str, container_name, client_options)); -} - -template -std::unique_ptr getAzureBlobStorageClientWithAuth( - const String & url, - const String & container_name, - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Azure::Storage::Blobs::BlobClientOptions & client_options) -{ - std::string connection_str; - if (config.has(config_prefix + ".connection_string")) - connection_str = config.getString(config_prefix + ".connection_string"); - - if (!connection_str.empty()) - return getClientWithConnectionString(connection_str, container_name, client_options); - - if (config.has(config_prefix + ".account_key") && config.has(config_prefix + ".account_name")) + if (isConnectionString(url)) { - auto storage_shared_key_credential = std::make_shared( - config.getString(config_prefix + ".account_name"), - config.getString(config_prefix + ".account_key") - ); - return std::make_unique(url, storage_shared_key_credential, client_options); + endpoint.storage_account_url = url; + auth_method = ConnectionString{url}; + return; } - if (config.getBool(config_prefix + ".use_workload_identity", false)) - { - auto workload_identity_credential = std::make_shared(); - return std::make_unique(url, workload_identity_credential); - } + size_t pos = url.find('?'); - auto managed_identity_credential = std::make_shared(); - return std::make_unique(url, managed_identity_credential, client_options); + /// If conneciton_url does not have '?', then its not SAS + if (pos == std::string::npos) + { + endpoint.storage_account_url = url; + auth_method = std::make_shared(); + } + else + { + endpoint.storage_account_url = url.substr(0, pos); + endpoint.sas_auth = url.substr(pos + 1); + auth_method = std::make_shared(); + } } -Azure::Storage::Blobs::BlobClientOptions getAzureBlobClientOptions(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +std::unique_ptr getContainerClient(const ConnectionParams & params, bool readonly) { - Azure::Core::Http::Policies::RetryOptions retry_options; - retry_options.MaxRetries = config.getUInt(config_prefix + ".max_tries", 10); - retry_options.RetryDelay = std::chrono::milliseconds(config.getUInt(config_prefix + ".retry_initial_backoff_ms", 10)); - retry_options.MaxRetryDelay = std::chrono::milliseconds(config.getUInt(config_prefix + ".retry_max_backoff_ms", 1000)); - - using CurlOptions = Azure::Core::Http::CurlTransportOptions; - CurlOptions curl_options; - curl_options.NoSignal = true; - - if (config.has(config_prefix + ".curl_ip_resolve")) - { - auto value = config.getString(config_prefix + ".curl_ip_resolve"); - if (value == "ipv4") - curl_options.IPResolve = CurlOptions::CURL_IPRESOLVE_V4; - else if (value == "ipv6") - curl_options.IPResolve = CurlOptions::CURL_IPRESOLVE_V6; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value for option 'curl_ip_resolve': {}. Expected one of 'ipv4' or 'ipv6'", value); - } - - Azure::Storage::Blobs::BlobClientOptions client_options; - client_options.Retry = retry_options; - client_options.Transport.Transport = std::make_shared(curl_options); - - client_options.ClickhouseOptions = Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}; - - return client_options; -} - -std::unique_ptr getAzureBlobContainerClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) -{ - auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix); - auto container_name = endpoint.container_name; - auto final_url = endpoint.getEndpoint(); - auto client_options = getAzureBlobClientOptions(config, config_prefix); - - if (endpoint.container_already_exists.value_or(false)) - return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix, client_options); - - auto blob_service_client = getAzureBlobStorageClientWithAuth(endpoint.getEndpointWithoutContainer(), container_name, config, config_prefix, client_options); + if (params.endpoint.container_already_exists.value_or(false) || readonly) + return params.createForContainer(); try { - return std::make_unique(blob_service_client->CreateBlobContainer(container_name).Value); + auto service_client = params.createForService(); + return std::make_unique(service_client->CreateBlobContainer(params.endpoint.container_name).Value); } catch (const Azure::Storage::StorageException & e) { /// If container_already_exists is not set (in config), ignore already exists error. /// (Conflict - The specified container already exists) - if (!endpoint.container_already_exists.has_value() && e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) - return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix, client_options); + if (!params.endpoint.container_already_exists.has_value() && e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) + return params.createForContainer(); throw; } } -std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +AuthMethod getAuthMethod(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - std::unique_ptr settings = std::make_unique(); + if (config.has(config_prefix + ".account_key") && config.has(config_prefix + ".account_name")) + { + return std::make_shared( + config.getString(config_prefix + ".account_name"), + config.getString(config_prefix + ".account_key") + ); + } + + if (config.has(config_prefix + ".connection_string")) + return ConnectionString{config.getString(config_prefix + ".connection_string")}; + + if (config.getBool(config_prefix + ".use_workload_identity", false)) + return std::make_shared(); + + return std::make_shared(); +} + +BlobClientOptions getClientOptions(const RequestSettings & settings, bool for_disk) +{ + Azure::Core::Http::Policies::RetryOptions retry_options; + retry_options.MaxRetries = static_cast(settings.sdk_max_retries); + retry_options.RetryDelay = std::chrono::milliseconds(settings.sdk_retry_initial_backoff_ms); + retry_options.MaxRetryDelay = std::chrono::milliseconds(settings.sdk_retry_max_backoff_ms); + + Azure::Core::Http::CurlTransportOptions curl_options; + curl_options.NoSignal = true; + curl_options.IPResolve = settings.curl_ip_resolve; + + Azure::Storage::Blobs::BlobClientOptions client_options; + client_options.Retry = retry_options; + client_options.Transport.Transport = std::make_shared(curl_options); + client_options.ClickhouseOptions = Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=for_disk}; + + return client_options; +} + +std::unique_ptr getRequestSettings(const Settings & query_settings) +{ + auto settings_ptr = std::make_unique(); + + settings_ptr->max_single_part_upload_size = query_settings.azure_max_single_part_upload_size; + settings_ptr->max_single_read_retries = query_settings.azure_max_single_read_retries; + settings_ptr->list_object_keys_size = static_cast(query_settings.azure_list_object_keys_size); + settings_ptr->sdk_max_retries = query_settings.azure_sdk_max_retries; + settings_ptr->sdk_retry_initial_backoff_ms = query_settings.azure_sdk_retry_initial_backoff_ms; + settings_ptr->sdk_retry_max_backoff_ms = query_settings.azure_sdk_retry_max_backoff_ms; + + return settings_ptr; +} + +std::unique_ptr getRequestSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +{ + auto settings = std::make_unique(); + settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", context->getSettings().azure_max_single_part_upload_size); settings->min_bytes_for_seek = config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024); settings->max_single_read_retries = config.getInt(config_prefix + ".max_single_read_retries", 3); @@ -262,10 +296,28 @@ std::unique_ptr getAzureBlobStorageSettings(const Po settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); + settings->sdk_max_retries = config.getUInt(config_prefix + ".max_tries", 10); + settings->sdk_retry_initial_backoff_ms = config.getUInt(config_prefix + ".retry_initial_backoff_ms", 10); + settings->sdk_retry_max_backoff_ms = config.getUInt(config_prefix + ".retry_max_backoff_ms", 1000); + + if (config.has(config_prefix + ".curl_ip_resolve")) + { + using CurlOptions = Azure::Core::Http::CurlTransportOptions; + + auto value = config.getString(config_prefix + ".curl_ip_resolve"); + if (value == "ipv4") + settings->curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_V4; + else if (value == "ipv6") + settings->curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_V6; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value for option 'curl_ip_resolve': {}. Expected one of 'ipv4' or 'ipv6'", value); + } return settings; } } +} + #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h new file mode 100644 index 00000000000..7e716adf4d0 --- /dev/null +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -0,0 +1,139 @@ +#pragma once + +#include +#include +#include "base/strong_typedef.h" +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace AzureBlobStorage +{ + +using ServiceClient = Azure::Storage::Blobs::BlobServiceClient; +using ContainerClient = Azure::Storage::Blobs::BlobContainerClient; +using BlobClient = Azure::Storage::Blobs::BlobClient; +using BlobClientOptions = Azure::Storage::Blobs::BlobClientOptions; + +struct RequestSettings +{ + RequestSettings() = default; + + size_t max_single_part_upload_size = 100 * 1024 * 1024; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset + uint64_t min_bytes_for_seek = 1024 * 1024; + size_t max_single_read_retries = 3; + size_t max_single_download_retries = 3; + int list_object_keys_size = 1000; + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t max_single_part_copy_size = 256 * 1024 * 1024; + bool use_native_copy = false; + size_t max_unexpected_write_error_retries = 4; + size_t max_inflight_parts_for_one_file = 20; + size_t strict_upload_part_size = 0; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t sdk_max_retries = 10; + size_t sdk_retry_initial_backoff_ms = 10; + size_t sdk_retry_max_backoff_ms = 1000; + + using CurlOptions = Azure::Core::Http::CurlTransportOptions; + CurlOptions::CurlOptIPResolve curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_WHATEVER; +}; + +struct Endpoint +{ + String storage_account_url; + String account_name; + String container_name; + String prefix; + String sas_auth; + std::optional container_already_exists; + + String getEndpoint() const + { + String url = storage_account_url; + if (url.ends_with('/')) + url.pop_back(); + + if (!account_name.empty()) + url += "/" + account_name; + + if (!container_name.empty()) + url += "/" + container_name; + + if (!prefix.empty()) + url += "/" + prefix; + + if (!sas_auth.empty()) + url += "?" + sas_auth; + + return url; + } + + String getEndpointWithoutContainer() const + { + String url = storage_account_url; + + if (!account_name.empty()) + url += "/" + account_name; + + if (!sas_auth.empty()) + url += "?" + sas_auth; + + return url; + } +}; + +using ConnectionString = StrongTypedef; + +using AuthMethod = std::variant< + ConnectionString, + std::shared_ptr, + std::shared_ptr, + std::shared_ptr>; + +struct ConnectionParams +{ + Endpoint endpoint; + AuthMethod auth_method; + BlobClientOptions client_options; + + String getContainer() const { return endpoint.container_name; } + String getConnectionURL() const; + + std::unique_ptr createForService() const; + std::unique_ptr createForContainer() const; +}; + +Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); +void processURL(const String & url, const String & container_name, Endpoint & endpoint, AuthMethod & auth_method); + +std::unique_ptr getContainerClient(const ConnectionParams & params, bool readonly); + +BlobClientOptions getClientOptions(const RequestSettings & settings, bool for_disk); +AuthMethod getAuthMethod(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); +std::unique_ptr getRequestSettings(const Settings & query_settings); +std::unique_ptr getRequestSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + +} + +} + +#endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 36225b13ee8..d0f39beb3ca 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include @@ -105,7 +105,7 @@ private: AzureObjectStorage::AzureObjectStorage( const String & name_, - AzureClientPtr && client_, + ClientPtr && client_, SettingsPtr && settings_, const String & object_namespace_) : name(name_) @@ -397,20 +397,37 @@ void AzureObjectStorage::copyObject( /// NOLINT void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { - auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); + auto new_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context); + bool is_client_for_disk = client.get()->GetClickhouseOptions().IsClientForDisk; + + AzureBlobStorage::ConnectionParams params + { + .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), + .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), + .client_options = AzureBlobStorage::getClientOptions(*new_settings, is_client_for_disk), + }; + + auto new_client = AzureBlobStorage::getContainerClient(params, /*readonly=*/ true); + settings.set(std::move(new_settings)); - /// We don't update client + client.set(std::move(new_client)); } std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { - return std::make_unique( - name, - getAzureBlobContainerClient(config, config_prefix), - getAzureBlobStorageSettings(config, config_prefix, context), - object_namespace - ); + auto new_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context); + bool is_client_for_disk = client.get()->GetClickhouseOptions().IsClientForDisk; + + AzureBlobStorage::ConnectionParams params + { + .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), + .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), + .client_options = AzureBlobStorage::getClientOptions(*new_settings, is_client_for_disk), + }; + + auto new_client = AzureBlobStorage::getContainerClient(params, /*readonly=*/ true); + return std::make_unique(name, std::move(new_client), std::move(new_settings), object_namespace); } } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f52ab803012..d8440453852 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -1,4 +1,5 @@ #pragma once +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -7,6 +8,7 @@ #include #include #include +#include namespace Poco { @@ -16,70 +18,15 @@ class Logger; namespace DB { -struct AzureObjectStorageSettings -{ - AzureObjectStorageSettings( - uint64_t max_single_part_upload_size_, - uint64_t min_bytes_for_seek_, - int max_single_read_retries_, - int max_single_download_retries_, - int list_object_keys_size_, - size_t min_upload_part_size_, - size_t max_upload_part_size_, - size_t max_single_part_copy_size_, - bool use_native_copy_, - size_t max_unexpected_write_error_retries_, - size_t max_inflight_parts_for_one_file_, - size_t strict_upload_part_size_, - size_t upload_part_size_multiply_factor_, - size_t upload_part_size_multiply_parts_count_threshold_) - : max_single_part_upload_size(max_single_part_upload_size_) - , min_bytes_for_seek(min_bytes_for_seek_) - , max_single_read_retries(max_single_read_retries_) - , max_single_download_retries(max_single_download_retries_) - , list_object_keys_size(list_object_keys_size_) - , min_upload_part_size(min_upload_part_size_) - , max_upload_part_size(max_upload_part_size_) - , max_single_part_copy_size(max_single_part_copy_size_) - , use_native_copy(use_native_copy_) - , max_unexpected_write_error_retries(max_unexpected_write_error_retries_) - , max_inflight_parts_for_one_file(max_inflight_parts_for_one_file_) - , strict_upload_part_size(strict_upload_part_size_) - , upload_part_size_multiply_factor(upload_part_size_multiply_factor_) - , upload_part_size_multiply_parts_count_threshold(upload_part_size_multiply_parts_count_threshold_) - { - } - - AzureObjectStorageSettings() = default; - - size_t max_single_part_upload_size = 100 * 1024 * 1024; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset - uint64_t min_bytes_for_seek = 1024 * 1024; - size_t max_single_read_retries = 3; - size_t max_single_download_retries = 3; - int list_object_keys_size = 1000; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t max_single_part_copy_size = 256 * 1024 * 1024; - bool use_native_copy = false; - size_t max_unexpected_write_error_retries = 4; - size_t max_inflight_parts_for_one_file = 20; - size_t strict_upload_part_size = 0; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; -}; - -using AzureClient = Azure::Storage::Blobs::BlobContainerClient; -using AzureClientPtr = std::unique_ptr; - class AzureObjectStorage : public IObjectStorage { public: - - using SettingsPtr = std::unique_ptr; + using ClientPtr = std::unique_ptr; + using SettingsPtr = std::unique_ptr; AzureObjectStorage( const String & name_, - AzureClientPtr && client_, + ClientPtr && client_, SettingsPtr && settings_, const String & object_namespace_); @@ -156,12 +103,8 @@ public: bool isRemote() const override { return true; } - std::shared_ptr getSettings() { return settings.get(); } - - std::shared_ptr getAzureBlobStorageClient() override - { - return client.get(); - } + std::shared_ptr getSettings() const { return settings.get(); } + std::shared_ptr getAzureBlobStorageClient() const override{ return client.get(); } private: using SharedAzureClientPtr = std::shared_ptr; @@ -169,8 +112,8 @@ private: const String name; /// client used to access the files in the Blob Storage cloud - MultiVersion client; - MultiVersion settings; + MultiVersion client; + MultiVersion settings; const String object_namespace; /// container + prefix LoggerPtr log; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 961c2709efc..60818933dec 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -120,7 +120,7 @@ public: const FileCacheSettings & getCacheSettings() const { return cache_settings; } #if USE_AZURE_BLOB_STORAGE - std::shared_ptr getAzureBlobStorageClient() override + std::shared_ptr getAzureBlobStorageClient() const override { return object_storage->getAzureBlobStorageClient(); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index eae31af9d44..27a58053752 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -238,7 +238,7 @@ public: virtual void setKeysGenerator(ObjectStorageKeysGeneratorPtr) { } #if USE_AZURE_BLOB_STORAGE - virtual std::shared_ptr getAzureBlobStorageClient() + virtual std::shared_ptr getAzureBlobStorageClient() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "This function is only implemented for AzureBlobStorage"); } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 7b949db268b..f8c1c564191 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -13,7 +13,7 @@ #endif #if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) #include -#include +#include #endif #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include @@ -293,12 +293,19 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) const ContextPtr & context, bool /* skip_access_check */) -> ObjectStoragePtr { - AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix); + auto azure_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context); + + AzureBlobStorage::ConnectionParams params + { + .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), + .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), + .client_options = AzureBlobStorage::getClientOptions(*azure_settings, /*for_disk=*/ true), + }; + return createObjectStorage( ObjectStorageType::Azure, config, config_prefix, name, - getAzureBlobContainerClient(config, config_prefix), - getAzureBlobStorageSettings(config, config_prefix, context), - endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix); + AzureBlobStorage::getContainerClient(params, /*readonly=*/ false), std::move(azure_settings), + params.endpoint.prefix.empty() ? params.endpoint.container_name : params.endpoint.container_name + "/" + params.endpoint.prefix); }; factory.registerObjectStorageType("azure_blob_storage", creator); factory.registerObjectStorageType("azure", creator); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 769f1a184f6..d648796b5df 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -44,7 +44,7 @@ namespace size_t total_size_, const String & dest_container_for_logging_, const String & dest_blob_, - std::shared_ptr settings_, + std::shared_ptr settings_, ThreadPoolCallbackRunnerUnsafe schedule_, const Poco::Logger * log_) : create_read_buffer(create_read_buffer_) @@ -69,7 +69,7 @@ namespace size_t total_size; const String & dest_container_for_logging; const String & dest_blob; - std::shared_ptr settings; + std::shared_ptr settings; ThreadPoolCallbackRunnerUnsafe schedule; const Poco::Logger * log; size_t max_single_part_upload_size; @@ -265,7 +265,7 @@ void copyDataToAzureBlobStorageFile( std::shared_ptr dest_client, const String & dest_container_for_logging, const String & dest_blob, - std::shared_ptr settings, + std::shared_ptr settings, ThreadPoolCallbackRunnerUnsafe schedule) { UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; @@ -282,11 +282,10 @@ void copyAzureBlobStorageFile( size_t size, const String & dest_container_for_logging, const String & dest_blob, - std::shared_ptr settings, + std::shared_ptr settings, const ReadSettings & read_settings, ThreadPoolCallbackRunnerUnsafe schedule) { - if (settings->use_native_copy) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 6ad54923ab5..73b91191b96 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -29,7 +29,7 @@ void copyAzureBlobStorageFile( size_t src_size, const String & dest_container_for_logging, const String & dest_blob, - std::shared_ptr settings, + std::shared_ptr settings, const ReadSettings & read_settings, ThreadPoolCallbackRunnerUnsafe schedule_ = {}); @@ -46,7 +46,7 @@ void copyDataToAzureBlobStorageFile( std::shared_ptr client, const String & dest_container_for_logging, const String & dest_blob, - std::shared_ptr settings, + std::shared_ptr settings, ThreadPoolCallbackRunnerUnsafe schedule_ = {}); } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index f2e2833dad4..d8fd5cbf05a 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,4 +1,6 @@ +#include #include +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #if USE_AZURE_BLOB_STORAGE #include @@ -95,43 +97,62 @@ const std::unordered_set optional_configuration_keys = { "storage_account_url", }; -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); } -} - -void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection) +void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection, const ContextPtr & local_context) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + String connection_url; + String container_name; + std::optional account_name; + std::optional account_key; + if (collection.has("connection_string")) - { - configuration.connection_url = collection.get("connection_string"); - configuration.is_connection_string = true; - } + connection_url = collection.get("connection_string"); + else if (collection.has("storage_account_url")) + connection_url = collection.get("storage_account_url"); - if (collection.has("storage_account_url")) - { - configuration.connection_url = collection.get("storage_account_url"); - configuration.is_connection_string = false; - } - - configuration.container = collection.get("container"); + container_name = collection.get("container"); configuration.blob_path = collection.get("blob_path"); if (collection.has("account_name")) - configuration.account_name = collection.get("account_name"); + account_name = collection.get("account_name"); if (collection.has("account_key")) - configuration.account_key = collection.get("account_key"); + account_key = collection.get("account_key"); configuration.structure = collection.getOrDefault("structure", "auto"); configuration.format = collection.getOrDefault("format", configuration.format); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + configuration.connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, local_context); } +AzureBlobStorage::ConnectionParams StorageAzureBlob::getConnectionParams( + const String & connection_url, + const String & container_name, + const std::optional & account_name, + const std::optional & account_key, + const ContextPtr & local_context) +{ + AzureBlobStorage::ConnectionParams connection_params; + auto request_settings = AzureBlobStorage::getRequestSettings(local_context->getSettingsRef()); + + if (account_name && account_key) + { + connection_params.endpoint.storage_account_url = connection_url; + connection_params.endpoint.container_name = container_name; + connection_params.auth_method = std::make_shared(*account_name, *account_key); + connection_params.client_options = AzureBlobStorage::getClientOptions(*request_settings, /*for_disk=*/ false); + } + else + { + AzureBlobStorage::processURL(connection_url, container_name, connection_params.endpoint, connection_params.auth_method); + connection_params.client_options = AzureBlobStorage::getClientOptions(*request_settings, /*for_disk=*/ false); + } + + return connection_params; +} StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, const ContextPtr & local_context) { @@ -144,8 +165,7 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) { - processNamedCollectionResult(configuration, *named_collection); - + processNamedCollectionResult(configuration, *named_collection, local_context); configuration.blobs_paths = {configuration.blob_path}; if (configuration.format == "auto") @@ -164,11 +184,12 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine std::unordered_map engine_args_to_idx; - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); + String connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + String container = checkAndGetLiteralArgument(engine_args[1], "container"); + configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blob_path"); - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + std::optional account_name; + std::optional account_key; auto is_format_arg = [] (const std::string & s) -> bool { @@ -198,8 +219,8 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine } else { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); } } else if (engine_args.size() == 6) @@ -211,12 +232,13 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine } else { - configuration.account_name = fourth_arg; + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; } } @@ -229,17 +251,20 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine } else { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); } } configuration.blobs_paths = {configuration.blob_path}; + configuration.connection_params = getConnectionParams(connection_url, container, account_name, account_key, local_context); if (configuration.format == "auto") configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); @@ -247,18 +272,6 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine return configuration; } - -AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(const ContextPtr & local_context) -{ - const auto & context_settings = local_context->getSettingsRef(); - auto settings_ptr = std::make_unique(); - settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; - settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); - - return settings_ptr; -} - void registerStorageAzureBlob(StorageFactory & factory) { factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) @@ -268,7 +281,8 @@ void registerStorageAzureBlob(StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext()); - auto client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client = AzureBlobStorage::getContainerClient(configuration.connection_params, /*readonly=*/ false); + // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current // session and user are ignored. @@ -299,11 +313,11 @@ void registerStorageAzureBlob(StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - auto settings = StorageAzureBlob::createSettings(args.getContext()); + auto azure_settings = AzureBlobStorage::getRequestSettings(args.getContext()->getSettingsRef()); return std::make_shared( - std::move(configuration), - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings),configuration.container), + configuration, + std::make_unique("AzureBlobStorage", std::move(client), std::move(azure_settings), configuration.connection_params.getContainer()), args.getContext(), args.table_id, args.columns, @@ -321,177 +335,6 @@ void registerStorageAzureBlob(StorageFactory & factory) }); } -static bool containerExists(std::unique_ptr &blob_service_client, std::string container_name) -{ - Azure::Storage::Blobs::ListBlobContainersOptions options; - options.Prefix = container_name; - options.PageSizeHint = 1; - - auto containers_list_response = blob_service_client->ListBlobContainers(options); - auto containers_list = containers_list_response.BlobContainers; - - for (const auto & container : containers_list) - { - if (container_name == container.Name) - return true; - } - return false; -} - -AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container) -{ - AzureClientPtr result; - - if (configuration.is_connection_string) - { - std::shared_ptr managed_identity_credential = std::make_shared(); - std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(configuration.connection_url)); - result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); - - if (attempt_to_create_container) - { - bool container_exists = containerExists(blob_service_client,configuration.container); - if (!container_exists) - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - - try - { - result->CreateIfNotExists(); - } - catch (const Azure::Storage::StorageException & e) - { - if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.")) - { - throw; - } - } - } - } - } - else - { - std::shared_ptr storage_shared_key_credential; - if (configuration.account_name.has_value() && configuration.account_key.has_value()) - { - storage_shared_key_credential - = std::make_shared(*configuration.account_name, *configuration.account_key); - } - - std::unique_ptr blob_service_client; - size_t pos = configuration.connection_url.find('?'); - std::shared_ptr managed_identity_credential; - if (storage_shared_key_credential) - { - blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); - } - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - blob_service_client = std::make_unique(configuration.connection_url, workload_identity_credential); - } - else - { - managed_identity_credential = std::make_shared(); - blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); - } - } - - std::string final_url; - if (pos != std::string::npos) - { - auto url_without_sas = configuration.connection_url.substr(0, pos); - final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container - + configuration.connection_url.substr(pos); - } - else - final_url - = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; - - if (!attempt_to_create_container) - { - if (storage_shared_key_credential) - return std::make_unique(final_url, storage_shared_key_credential); - else - return std::make_unique(final_url, managed_identity_credential); - } - - bool container_exists = containerExists(blob_service_client,configuration.container); - if (container_exists) - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - result = std::make_unique(final_url, workload_identity_credential); - } - else - result = std::make_unique(final_url, managed_identity_credential); - } - } - else - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - try - { - result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); - } - catch (const Azure::Storage::StorageException & e) - { - if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.") - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - result = std::make_unique(final_url, workload_identity_credential); - } - else - result = std::make_unique(final_url, managed_identity_credential); - } - } - else - { - throw; - } - } - } - } - - return result; -} - -Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const -{ - if (!is_connection_string) - return Poco::URI(connection_url); - - auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); - return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); -} - - StorageAzureBlob::StorageAzureBlob( const Configuration & configuration_, std::unique_ptr && object_storage_, @@ -513,7 +356,8 @@ StorageAzureBlob::StorageAzureBlob( { if (configuration.format != "auto") FormatFactory::instance().checkFormatName(configuration.format); - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.getConnectionURL()); + + context->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.connection_params.getConnectionURL())); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) @@ -850,13 +694,13 @@ void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate) { /// Iterate through disclosed globs and make a source for each file iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blob_path, + storage->object_storage.get(), configuration.connection_params.getContainer(), configuration.blob_path, predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); } else { iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blobs_paths, + storage->object_storage.get(), configuration.connection_params.getContainer(), configuration.blobs_paths, predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); } } @@ -879,8 +723,8 @@ void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, cons max_block_size, configuration.compression_method, storage->object_storage.get(), - configuration.container, - configuration.connection_url, + configuration.connection_params.getContainer(), + configuration.connection_params.endpoint.storage_account_url, iterator_wrapper, need_only_count)); } @@ -1455,7 +1299,8 @@ namespace if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure) return; - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; + const auto & params = configuration.connection_params; + String source = fs::path(params.getConnectionURL()) / params.getContainer() / current_path_with_metadata.relative_path; auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows); } @@ -1466,7 +1311,8 @@ namespace || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) return; - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; + const auto & params = configuration.connection_params; + String source = fs::path(params.getConnectionURL()) / params.getContainer() / current_path_with_metadata.relative_path; auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns); } @@ -1477,7 +1323,9 @@ namespace || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) return; - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + const auto & params = configuration.connection_params; + auto host_and_bucket = params.getConnectionURL() + '/' + params.getContainer(); + Strings sources; sources.reserve(read_keys.size()); std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); @@ -1520,8 +1368,10 @@ namespace return std::nullopt; }; - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + const auto & params = configuration.connection_params; + auto host_and_bucket = params.getConnectionURL() + '/' + params.getContainer(); String source = host_and_bucket + '/' + it->relative_path; + if (format) { auto cache_key = getKeyForSchemaCache(source, *format, format_settings, context); @@ -1573,12 +1423,12 @@ std::pair StorageAzureBlob::getTableStructureAndForm if (configuration.withGlobs()) { file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blob_path, nullptr, NamesAndTypesList{}, ctx, &read_keys); + object_storage, configuration.connection_params.getContainer(), configuration.blob_path, nullptr, NamesAndTypesList{}, ctx, &read_keys); } else { file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys); + object_storage, configuration.connection_params.getContainer(), configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys); } ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, format, configuration, format_settings, read_keys, ctx); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 20e7f4a6c90..affa02928b6 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -1,5 +1,7 @@ #pragma once +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" +#include "Interpreters/Context_fwd.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -37,21 +39,13 @@ public: bool withWildcard() const { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + static constexpr auto PARTITION_ID_WILDCARD = "{_partition_id}"; return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; } - Poco::URI getConnectionURL() const; - - std::string connection_url; - bool is_connection_string; - - std::optional account_name; - std::optional account_key; - - std::string container; std::string blob_path; std::vector blobs_paths; + AzureBlobStorage::ConnectionParams connection_params; }; StorageAzureBlob( @@ -67,16 +61,10 @@ public: ASTPtr partition_by_); static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context); - static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container = true); + static AzureBlobStorage::ConnectionParams getConnectionParams(const String & connection_url, const String & container_name, const std::optional & account_name, const std::optional & account_key, const ContextPtr & local_context); + static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection, const ContextPtr & local_context); - static AzureObjectStorage::SettingsPtr createSettings(const ContextPtr & local_context); - - static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); - - String getName() const override - { - return name; - } + String getName() const override { return name; } void read( QueryPlan & query_plan, diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index a80d121567a..6f6ae8763fd 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -1,4 +1,5 @@ #include "Storages/StorageAzureBlobCluster.h" +#include #include "config.h" @@ -41,7 +42,7 @@ StorageAzureBlobCluster::StorageAzureBlobCluster( , configuration{configuration_} , object_storage(std::move(object_storage_)) { - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL()); + context->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration_.connection_params.getConnectionURL())); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) @@ -79,7 +80,7 @@ void StorageAzureBlobCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, cons RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { auto iterator = std::make_shared( - object_storage.get(), configuration.container, configuration.blob_path, + object_storage.get(), configuration.connection_params.getContainer(), configuration.blob_path, predicate, getVirtualsList(), context, nullptr); auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 275cd2a9cbb..c471a72d8c7 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -35,16 +35,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace -{ - -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); -} - -} - void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) { /// Supported signatures: @@ -54,7 +44,7 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) { - StorageAzureBlob::processNamedCollectionResult(configuration, *named_collection); + StorageAzureBlob::processNamedCollectionResult(configuration, *named_collection, local_context); configuration.blobs_paths = {configuration.blob_path}; @@ -74,14 +64,14 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const std::unordered_map engine_args_to_idx; - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); + String connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + String container = checkAndGetLiteralArgument(engine_args[1], "container"); configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - auto is_format_arg - = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().exists(s); }; + std::optional account_name; + std::optional account_key; + + auto is_format_arg = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().exists(s); }; if (engine_args.size() == 4) { @@ -105,8 +95,8 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const } else { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); } } else if (engine_args.size() == 6) @@ -120,8 +110,9 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const } else { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name/structure"); if (is_format_arg(sixth_arg)) configuration.format = sixth_arg; @@ -132,28 +123,33 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const else if (engine_args.size() == 7) { auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); } else if (engine_args.size() == 8) { auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); } configuration.blobs_paths = {configuration.blob_path}; + configuration.connection_params = StorageAzureBlob::getConnectionParams(connection_url, container, account_name, account_key, local_context); if (configuration.format == "auto") configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); @@ -330,12 +326,19 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex if (configuration.structure == "auto") { context->checkAccess(getSourceAccessType()); - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container); + auto client = AzureBlobStorage::getContainerClient(configuration.connection_params, !is_insert_query); + auto settings = AzureBlobStorage::getRequestSettings(context->getSettingsRef()); + + auto object_storage = std::make_unique( + "AzureBlobStorageTableFunction", + std::move(client), + std::move(settings), + configuration.connection_params.getContainer()); + if (configuration.format == "auto") return StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, std::nullopt, context).first; + return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); } @@ -354,8 +357,8 @@ std::unordered_set TableFunctionAzureBlobStorage::getVirtualsToCheckBefo StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); + auto client = AzureBlobStorage::getContainerClient(configuration.connection_params, !is_insert_query); + auto settings = AzureBlobStorage::getRequestSettings(context->getSettingsRef()); ColumnsDescription columns; if (configuration.structure != "auto") @@ -365,7 +368,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct StoragePtr storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.connection_params.getContainer()), context, StorageID(getDatabaseName(), table_name), columns, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index 04dddca7672..fb311c74657 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -1,3 +1,4 @@ +#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -31,30 +32,30 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( columns = structure_hint; } - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); + auto settings = AzureBlobStorage::getRequestSettings(context->getSettingsRef()); + auto client = AzureBlobStorage::getContainerClient(configuration.connection_params, !is_insert_query); if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { /// On worker node this filename won't contains globs storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.connection_params.getContainer()), context, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings - /* distributed_processing */ true, - /*partition_by_=*/nullptr); + /*comment=*/ String{}, + /*format_settings=*/ std::nullopt, /// No format_settings + /*distributed_processing=*/ true, + /*partition_by=*/ nullptr); } else { storage = std::make_shared( cluster_name, configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.connection_params.getContainer()), StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, From de3d95a7f05156330dc3cbad3dee7265a027b074 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 11 May 2024 21:00:08 +0000 Subject: [PATCH 017/107] fix style check --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a04d7f54884..1776163688e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -117,8 +117,8 @@ class IColumn; M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(UInt64, azure_sdk_max_retries, 10, "Maximum number of retries in azure sdk", 0) \ - M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff beetween retries in azure sdk", 0) \ - M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff beetween retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff between retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff between retries in azure sdk", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ From 9ef86e948ecb1408d6ce8df2e2602584d591252c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 13 May 2024 16:07:28 +0000 Subject: [PATCH 018/107] fix tests --- src/Backups/BackupIO_AzureBlobStorage.cpp | 1 - src/Backups/BackupIO_AzureBlobStorage.h | 5 +---- .../registerBackupEngineAzureBlobStorage.cpp | 3 +-- .../AzureBlobStorageCommon.cpp | 4 ---- .../AzureBlobStorage/AzureBlobStorageCommon.h | 6 +----- src/Storages/StorageAzureBlob.cpp | 19 +++++++------------ src/Storages/StorageAzureBlob.h | 3 --- .../TableFunctionAzureBlobStorageCluster.cpp | 4 ---- .../test_storage_azure_blob_storage/test.py | 2 +- 9 files changed, 11 insertions(+), 36 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 3f60ed5c0b4..6ae67ad5dfc 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 0829c3258c9..8f0a6e8fb5d 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -1,13 +1,10 @@ #pragma once - -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE #include #include -#include -#include +#include namespace DB diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 6974d16e2f6..98920d80662 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -1,4 +1,3 @@ -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #include @@ -6,7 +5,7 @@ #if USE_AZURE_BLOB_STORAGE #include -#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 76054efff19..a39cc89b93b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -1,8 +1,4 @@ -#include -#include -#include #include -#include #if USE_AZURE_BLOB_STORAGE diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index 7e716adf4d0..5f9f280ad4a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -1,8 +1,4 @@ #pragma once - -#include -#include -#include "base/strong_typedef.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -15,10 +11,10 @@ #include #include -#include #include #include #include +#include namespace DB { diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 2341b8dc94e..a82de72af6d 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,6 +1,4 @@ -#include #include -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #if USE_AZURE_BLOB_STORAGE #include @@ -44,6 +42,7 @@ #include #include +#include #include @@ -1307,8 +1306,7 @@ namespace if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure) return; - const auto & params = configuration.connection_params; - String source = fs::path(params.getConnectionURL()) / params.getContainer() / current_path_with_metadata.relative_path; + String source = fs::path(configuration.connection_params.endpoint.getEndpoint()) / current_path_with_metadata.relative_path; auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows); } @@ -1319,8 +1317,7 @@ namespace || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) return; - const auto & params = configuration.connection_params; - String source = fs::path(params.getConnectionURL()) / params.getContainer() / current_path_with_metadata.relative_path; + String source = fs::path(configuration.connection_params.endpoint.getEndpoint()) / current_path_with_metadata.relative_path; auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns); } @@ -1331,12 +1328,11 @@ namespace || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) return; - const auto & params = configuration.connection_params; - auto host_and_bucket = params.getConnectionURL() + '/' + params.getContainer(); + auto endpoint = fs::path(configuration.connection_params.endpoint.getEndpoint()); Strings sources; sources.reserve(read_keys.size()); - std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); + std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem) { return endpoint / elem.relative_path; }); auto cache_keys = getKeysForSchemaCache(sources, *format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); } @@ -1376,9 +1372,8 @@ namespace return std::nullopt; }; - const auto & params = configuration.connection_params; - auto host_and_bucket = params.getConnectionURL() + '/' + params.getContainer(); - String source = host_and_bucket + '/' + it->relative_path; + auto endpoint = fs::path(configuration.connection_params.endpoint.getEndpoint()); + String source = endpoint / it->relative_path; if (format) { diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 936f32c3cb0..396934b4212 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -1,7 +1,4 @@ #pragma once - -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" -#include "Interpreters/Context_fwd.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index fb311c74657..d72735bb47b 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -1,4 +1,3 @@ -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -7,11 +6,8 @@ #include #include #include - #include "registerTableFunctions.h" -#include - namespace DB { diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 78aaf26a2a7..c3204808d6f 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -789,7 +789,7 @@ def test_read_subcolumns(cluster): def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = ( - f"select * from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont_not_exists', 'test_table.csv', " + f"select * from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont-not-exists', 'test_table.csv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" ) expected_err_msg = "container does not exist" From 35038f2458f2642700ee685a5c4f43729228d8a0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 May 2024 14:23:24 +0000 Subject: [PATCH 019/107] rename parameter --- .../test_backup_restore_azure_blob_storage/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 1a1458cb68e..6765a519a6d 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -30,12 +30,12 @@ def generate_cluster_def(port): DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:{port}/devstoreaccount1; - cont + cont CSV http://azurite1:{port}/devstoreaccount1 - cont + cont CSV devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== From 29250418cb5d666b81043695a1df09e4df94e539 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 17 May 2024 20:43:40 +0000 Subject: [PATCH 020/107] fix backward incompatibility --- .../AzureBlobStorage/AzureBlobStorageCommon.cpp | 15 +++++++++++++-- .../test.py | 4 ++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index a39cc89b93b..11253d25e3d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -92,6 +92,17 @@ Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const String container_name; String prefix; + auto get_container_name = [&] + { + if (config.has(config_prefix + ".container_name")) + return config.getString(config_prefix + ".container_name"); + + if (config.has(config_prefix + ".container")) + return config.getString(config_prefix + ".container"); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected either `container` or `container_name` parameter in config"); + }; + if (config.has(config_prefix + ".endpoint")) { String endpoint = config.getString(config_prefix + ".endpoint"); @@ -154,13 +165,13 @@ Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const else if (config.has(config_prefix + ".connection_string")) { storage_url = config.getString(config_prefix + ".connection_string"); - container_name = config.getString(config_prefix + ".container_name"); + container_name = get_container_name(); } else if (config.has(config_prefix + ".storage_account_url")) { storage_url = config.getString(config_prefix + ".storage_account_url"); validateStorageAccountUrl(storage_url); - container_name = config.getString(config_prefix + ".container_name"); + container_name = get_container_name(); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected either `storage_account_url` or `connection_string` or `endpoint` in config"); diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 6765a519a6d..1a1458cb68e 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -30,12 +30,12 @@ def generate_cluster_def(port): DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:{port}/devstoreaccount1; - cont + cont CSV http://azurite1:{port}/devstoreaccount1 - cont + cont CSV devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== From 90faa7b1eced87e0f2979e792d6569d1e2e005e8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 19 Jun 2024 21:44:37 +0100 Subject: [PATCH 021/107] impl --- src/Common/CgroupsMemoryUsageObserver.cpp | 178 +++++++++++----------- src/Common/CgroupsMemoryUsageObserver.h | 14 +- 2 files changed, 102 insertions(+), 90 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 8a4792f0a5a..20db6a64a31 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -1,3 +1,5 @@ +#include +#include #include #if defined(OS_LINUX) @@ -28,8 +30,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_CLOSE_FILE; - extern const int CANNOT_OPEN_FILE; extern const int FILE_DOESNT_EXIST; extern const int INCORRECT_DATA; } @@ -107,6 +107,75 @@ void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmo namespace { +/// Format is +/// kernel 5 +/// rss 15 +/// [...] +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +{ + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t mem_usage = 0; + readIntText(mem_usage, buf); + return mem_usage; + } + + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); +} + +struct CgroupsV1Reader : ICgroupsReader +{ + CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return readMetricFromStatFile(buf, "rss"); + } + +private: + std::mutex mutex; + ReadBufferFromFile buf TSA_GUARDED_BY(mutex); +}; + +struct CgroupsV2Reader : ICgroupsReader +{ + CgroupsV2Reader(const std::filesystem::path & stat_file_dir) + : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") + { + } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + current_buf.rewind(); + stat_buf.rewind(); + + uint64_t mem_usage = 0; + /// memory.current contains a single number + readIntText(mem_usage, current_buf); + mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file"); + return mem_usage; + } + +private: + std::mutex mutex; + ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex); + ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); +}; + /// Caveats: /// - All of the logic in this file assumes that the current process is the only process in the /// containing cgroup (or more precisely: the only process with significant memory consumption). @@ -117,7 +186,7 @@ namespace /// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such /// systems existed only for a short transition period. -std::optional getCgroupsV2FileName() +std::optional getCgroupsV2Path() { if (!cgroupsV2Enabled()) return {}; @@ -132,29 +201,30 @@ std::optional getCgroupsV2FileName() /// level, try again at the parent level as memory settings are inherited. while (current_cgroup != default_cgroups_mount.parent_path()) { - auto path = current_cgroup / "memory.current"; - if (std::filesystem::exists(path)) - return {path}; + const auto current_path = current_cgroup / "memory.current"; + const auto stat_path = current_cgroup / "memory.stat"; + if (std::filesystem::exists(current_path) && std::filesystem::exists(stat_path)) + return {current_cgroup}; current_cgroup = current_cgroup.parent_path(); } return {}; } -std::optional getCgroupsV1FileName() +std::optional getCgroupsV1Path() { auto path = default_cgroups_mount / "memory/memory.stat"; if (!std::filesystem::exists(path)) return {}; - return {path}; + return {default_cgroups_mount / "memory"}; } -std::pair getCgroupsFileName() +std::pair getCgroupsPath() { - auto v2_file_name = getCgroupsV2FileName(); + auto v2_file_name = getCgroupsV2Path(); if (v2_file_name.has_value()) return {*v2_file_name, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; - auto v1_file_name = getCgroupsV1FileName(); + auto v1_file_name = getCgroupsV1Path(); if (v1_file_name.has_value()) return {*v1_file_name, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; @@ -166,87 +236,25 @@ std::pair getCgroupsFil CgroupsMemoryUsageObserver::MemoryUsageFile::MemoryUsageFile(LoggerPtr log_) : log(log_) { - std::tie(file_name, version) = getCgroupsFileName(); + const auto [cgroup_path, version] = getCgroupsPath(); - LOG_INFO(log, "Will read the current memory usage from '{}' (cgroups version: {})", file_name, (version == CgroupsVersion::V1) ? "v1" : "v2"); + if (version == CgroupsVersion::V2) + cgroup_reader = std::make_unique(cgroup_path); + else + cgroup_reader = std::make_unique(cgroup_path); - fd = ::open(file_name.data(), O_RDONLY); - if (fd == -1) - ErrnoException::throwFromPath( - (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, - file_name, "Cannot open file '{}'", file_name); -} - -CgroupsMemoryUsageObserver::MemoryUsageFile::~MemoryUsageFile() -{ - assert(fd != -1); - if (::close(fd) != 0) - { - try - { - ErrnoException::throwFromPath( - ErrorCodes::CANNOT_CLOSE_FILE, - file_name, "Cannot close file '{}'", file_name); - } - catch (const ErrnoException &) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - } - } + LOG_INFO( + log, + "Will read the current memory usage from '{}' (cgroups version: {})", + cgroup_path, + (version == CgroupsVersion::V1) ? "v1" : "v2"); } uint64_t CgroupsMemoryUsageObserver::MemoryUsageFile::readMemoryUsage() const { - /// File read is probably not read is thread-safe, just to be sure - std::lock_guard lock(mutex); - - ReadBufferFromFileDescriptor buf(fd); - buf.rewind(); - - uint64_t mem_usage = 0; - - switch (version) - { - case CgroupsVersion::V1: - { - /// Format is - /// kernel 5 - /// rss 15 - /// [...] - std::string key; - bool found_rss = false; - - while (!buf.eof()) - { - readStringUntilWhitespace(key, buf); - if (key != "rss") - { - std::string dummy; - readStringUntilNewlineInto(dummy, buf); - buf.ignore(); - continue; - } - - assertChar(' ', buf); - readIntText(mem_usage, buf); - found_rss = true; - break; - } - - if (!found_rss) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find 'rss' in '{}'", file_name); - - break; - } - case CgroupsVersion::V2: - { - readIntText(mem_usage, buf); - break; - } - } - + chassert(cgroup_reader); + const auto mem_usage = cgroup_reader->readMemoryUsage(); LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(mem_usage)); - return mem_usage; } diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index edc1cee750a..62bbabb9e86 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -3,11 +3,19 @@ #include #include +#include #include namespace DB { +struct ICgroupsReader +{ + virtual ~ICgroupsReader() = default; + + virtual uint64_t readMemoryUsage() = 0; +}; + /// Does two things: /// 1. Periodically reads the memory usage of the process from Linux cgroups. /// You can specify soft or hard memory limits: @@ -66,14 +74,10 @@ private: { public: explicit MemoryUsageFile(LoggerPtr log_); - ~MemoryUsageFile(); uint64_t readMemoryUsage() const; private: LoggerPtr log; - mutable std::mutex mutex; - int fd TSA_GUARDED_BY(mutex) = -1; - CgroupsVersion version; - std::string file_name; + std::unique_ptr cgroup_reader; }; MemoryUsageFile memory_usage_file; From 1fa5212836219f89fe2ea8877d882daf0a928bce Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 17:49:51 +0100 Subject: [PATCH 022/107] remove MemoryUsageFile --- src/Common/CgroupsMemoryUsageObserver.cpp | 153 +++++++++++----------- src/Common/CgroupsMemoryUsageObserver.h | 15 +-- 2 files changed, 75 insertions(+), 93 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 20db6a64a31..23bfec4322b 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -24,84 +24,17 @@ #define STRINGIFY(x) STRINGIFY_HELPER(x) #endif +using namespace DB; namespace DB { namespace ErrorCodes { - extern const int FILE_DOESNT_EXIST; - extern const int INCORRECT_DATA; +extern const int FILE_DOESNT_EXIST; +extern const int INCORRECT_DATA; } -CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) - : log(getLogger("CgroupsMemoryUsageObserver")) - , wait_time(wait_time_) - , memory_usage_file(log) -{ - LOG_INFO(log, "Initialized cgroups memory limit observer, wait time is {} sec", wait_time.count()); -} - -CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() -{ - stopThread(); -} - -void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_) -{ - std::lock_guard limit_lock(limit_mutex); - - if (hard_limit_ == hard_limit && soft_limit_ == soft_limit) - return; - - hard_limit = hard_limit_; - soft_limit = soft_limit_; - - on_hard_limit = [this, hard_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_)); - - /// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are - /// really zero. Trying to avoid OOM ... - MemoryTracker::setRSS(hard_limit_, 0); - } - else - { - LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_)); - } - }; - - on_soft_limit = [this, soft_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); - -#if USE_JEMALLOC - LOG_INFO(log, "Purging jemalloc arenas"); - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); -#endif - /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. - uint64_t memory_usage = memory_usage_file.readMemoryUsage(); - MemoryTracker::setRSS(memory_usage, 0); - - LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); - } - else - { - LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_)); - } - }; - - LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_)); -} - -void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_) -{ - std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); - on_memory_amount_available_changed = on_memory_amount_available_changed_; } namespace @@ -233,8 +166,11 @@ std::pair getCgroupsPat } -CgroupsMemoryUsageObserver::MemoryUsageFile::MemoryUsageFile(LoggerPtr log_) - : log(log_) +namespace DB +{ + +CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) { const auto [cgroup_path, version] = getCgroupsPath(); @@ -245,17 +181,73 @@ CgroupsMemoryUsageObserver::MemoryUsageFile::MemoryUsageFile(LoggerPtr log_) LOG_INFO( log, - "Will read the current memory usage from '{}' (cgroups version: {})", + "Will read the current memory usage from '{}' (cgroups version: {}), wait time is {} sec", cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2"); + (version == CgroupsVersion::V1) ? "v1" : "v2", + wait_time.count()); } -uint64_t CgroupsMemoryUsageObserver::MemoryUsageFile::readMemoryUsage() const +CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() { - chassert(cgroup_reader); - const auto mem_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(mem_usage)); - return mem_usage; + stopThread(); +} + +void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_) +{ + std::lock_guard limit_lock(limit_mutex); + + if (hard_limit_ == hard_limit && soft_limit_ == soft_limit) + return; + + hard_limit = hard_limit_; + soft_limit = soft_limit_; + + on_hard_limit = [this, hard_limit_](bool up) + { + if (up) + { + LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_)); + + /// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are + /// really zero. Trying to avoid OOM ... + MemoryTracker::setRSS(hard_limit_, 0); + } + else + { + LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_)); + } + }; + + on_soft_limit = [this, soft_limit_](bool up) + { + if (up) + { + LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); + +# if USE_JEMALLOC + LOG_INFO(log, "Purging jemalloc arenas"); + mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); +# endif + /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. + uint64_t memory_usage = cgroup_reader->readMemoryUsage(); + LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(memory_usage)); + MemoryTracker::setRSS(memory_usage, 0); + + LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); + } + else + { + LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_)); + } + }; + + LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_)); +} + +void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_) +{ + std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); + on_memory_amount_available_changed = on_memory_amount_available_changed_; } void CgroupsMemoryUsageObserver::startThread() @@ -309,7 +301,8 @@ void CgroupsMemoryUsageObserver::runThread() std::lock_guard limit_lock(limit_mutex); if (soft_limit > 0 && hard_limit > 0) { - uint64_t memory_usage = memory_usage_file.readMemoryUsage(); + uint64_t memory_usage = cgroup_reader->readMemoryUsage(); + LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(memory_usage)); if (memory_usage > hard_limit) { if (last_memory_usage <= hard_limit) diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 62bbabb9e86..b848a2bff3c 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -69,23 +69,12 @@ private: uint64_t last_memory_usage = 0; /// how much memory does the process use uint64_t last_available_memory_amount; /// how much memory can the process use - /// Represents the cgroup virtual file that shows the memory consumption of the process's cgroup. - struct MemoryUsageFile - { - public: - explicit MemoryUsageFile(LoggerPtr log_); - uint64_t readMemoryUsage() const; - private: - LoggerPtr log; - std::unique_ptr cgroup_reader; - }; - - MemoryUsageFile memory_usage_file; - void stopThread(); void runThread(); + std::unique_ptr cgroup_reader; + std::mutex thread_mutex; std::condition_variable cond; ThreadFromGlobalPool thread; From 0f0e1cee63a8e2047f2092d7db4e81c5a3b53572 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 20:13:59 +0100 Subject: [PATCH 023/107] fix tidy --- src/Common/CgroupsMemoryUsageObserver.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 23bfec4322b..c37e3c74db9 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -1,5 +1,5 @@ +#include #include -#include #include #if defined(OS_LINUX) @@ -69,7 +69,7 @@ uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & ke struct CgroupsV1Reader : ICgroupsReader { - CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + explicit CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } uint64_t readMemoryUsage() override { @@ -85,7 +85,7 @@ private: struct CgroupsV2Reader : ICgroupsReader { - CgroupsV2Reader(const std::filesystem::path & stat_file_dir) + explicit CgroupsV2Reader(const std::filesystem::path & stat_file_dir) : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") { } From 7e0ed1b02cb55b0ce5788c9f1cf7c69c163ad14b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 21 Jun 2024 21:29:46 +0100 Subject: [PATCH 024/107] add test --- src/Common/CgroupsMemoryUsageObserver.cpp | 4 +-- .../test_memory_limit_observer/test.py | 25 ++++++++++++++++++- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index c37e3c74db9..33393a8b9c6 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -230,7 +230,7 @@ void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint # endif /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(memory_usage)); + LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); MemoryTracker::setRSS(memory_usage, 0); LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); @@ -302,7 +302,7 @@ void CgroupsMemoryUsageObserver::runThread() if (soft_limit > 0 && hard_limit > 0) { uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} from cgroups", ReadableSize(memory_usage)); + LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); if (memory_usage > hard_limit) { if (last_memory_usage <= hard_limit) diff --git a/tests/integration/test_memory_limit_observer/test.py b/tests/integration/test_memory_limit_observer/test.py index fe3acd9a0cf..369b9241f07 100644 --- a/tests/integration/test_memory_limit_observer/test.py +++ b/tests/integration/test_memory_limit_observer/test.py @@ -35,7 +35,7 @@ def get_latest_mem_limit(): ).strip() ) return mem_limit - except Exception as e: + except Exception: time.sleep(1) raise Exception("Cannot get memory limit") @@ -51,3 +51,26 @@ def test_observe_memory_limit(started_cluster): if new_max_mem > original_max_mem: return raise Exception("the memory limit does not increase as expected") + + +def test_memory_usage_doesnt_include_page_cache_size(started_cluster): + # populate page cache with 10GB of data + node1.exec_in_container( + ["dd", "if=/dev/zero", "of=outputfile", "bs=1M", "count=10K"] + ) + + observer_refresh_period = int( + node1.query( + "select value from system.server_settings where name = 'cgroups_memory_usage_observer_wait_time'" + ).strip() + ) + time.sleep(observer_refresh_period + 1) + + max_mem_usage_from_cgroup = node1.query( + """ + SELECT max(toUInt64(replaceRegexpAll(message, 'Read current memory usage (\\d+) bytes.*', '\\1'))) AS max_mem + FROM system.text_log + WHERE logger_name = 'CgroupsMemoryUsageObserver' AND message LIKE 'Read current memory usage%bytes%' + """ + ).strip() + assert int(max_mem_usage_from_cgroup) < 2 * 2 ** 30 From 750c902671bb64f02c7bf6918779561a06711de6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jun 2024 20:44:53 +0000 Subject: [PATCH 025/107] Automatic style fix --- tests/integration/test_memory_limit_observer/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_memory_limit_observer/test.py b/tests/integration/test_memory_limit_observer/test.py index 369b9241f07..d8c2a0e8ad7 100644 --- a/tests/integration/test_memory_limit_observer/test.py +++ b/tests/integration/test_memory_limit_observer/test.py @@ -73,4 +73,4 @@ def test_memory_usage_doesnt_include_page_cache_size(started_cluster): WHERE logger_name = 'CgroupsMemoryUsageObserver' AND message LIKE 'Read current memory usage%bytes%' """ ).strip() - assert int(max_mem_usage_from_cgroup) < 2 * 2 ** 30 + assert int(max_mem_usage_from_cgroup) < 2 * 2**30 From e20136ce25f85d463f4e3a033ac0a2a1d97431e5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jun 2024 17:29:41 +0100 Subject: [PATCH 026/107] fix test --- tests/integration/test_memory_limit_observer/test.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_memory_limit_observer/test.py b/tests/integration/test_memory_limit_observer/test.py index d8c2a0e8ad7..f19e119c019 100644 --- a/tests/integration/test_memory_limit_observer/test.py +++ b/tests/integration/test_memory_limit_observer/test.py @@ -54,10 +54,13 @@ def test_observe_memory_limit(started_cluster): def test_memory_usage_doesnt_include_page_cache_size(started_cluster): - # populate page cache with 10GB of data - node1.exec_in_container( - ["dd", "if=/dev/zero", "of=outputfile", "bs=1M", "count=10K"] - ) + try: + # populate page cache with 10GB of data; it might be killed by OOM killer but it is fine + node1.exec_in_container( + ["dd", "if=/dev/zero", "of=outputfile", "bs=1M", "count=10K"] + ) + except Exception: + pass observer_refresh_period = int( node1.query( From b1f87c578161578a03ee99ab87899cf4deb2c2ef Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jun 2024 21:04:04 +0000 Subject: [PATCH 027/107] return back settings and fix build --- src/Backups/BackupIO_AzureBlobStorage.cpp | 17 +++-- src/Core/Settings.h | 6 ++ .../AzureBlobStorageCommon.cpp | 68 ++++++++++++------- .../AzureBlobStorage/AzureBlobStorageCommon.h | 9 ++- .../ObjectStorages/ObjectStorageFactory.cpp | 3 +- .../ObjectStorage/Azure/Configuration.cpp | 12 +++- 6 files changed, 74 insertions(+), 41 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 596c308ca8a..0ee0160a969 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -41,18 +41,17 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , blob_path(blob_path_) { auto client_ptr = AzureBlobStorage::getContainerClient(connection_params, /*readonly=*/ false); + auto settings_ptr = AzureBlobStorage::getRequestSettingsForBackup(context_->getSettingsRef(), allow_azure_native_copy); object_storage = std::make_unique( "BackupReaderAzureBlobStorage", std::move(client_ptr), - AzureBlobStorage::getRequestSettings(context_->getSettingsRef()), + std::move(settings_ptr), connection_params.getContainer(), connection_params.getConnectionURL()); client = object_storage->getAzureBlobStorageClient(); - auto settings_copy = *object_storage->getSettings(); - settings_copy.use_native_copy = allow_azure_native_copy; - settings = std::make_unique(settings_copy); + settings = object_storage->getSettings(); } BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; @@ -122,8 +121,8 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const AzureBlobStorage::ConnectionParams & connection_params_, - bool allow_azure_native_copy, const String & blob_path_, + bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, @@ -137,17 +136,17 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( connection_params.endpoint.container_already_exists = true; auto client_ptr = AzureBlobStorage::getContainerClient(connection_params, /*readonly=*/ false); + auto settings_ptr = AzureBlobStorage::getRequestSettingsForBackup(context_->getSettingsRef(), allow_azure_native_copy); + object_storage = std::make_unique( "BackupWriterAzureBlobStorage", std::move(client_ptr), - AzureBlobStorage::getRequestSettings(context_->getSettingsRef()), + std::move(settings_ptr), connection_params.getContainer(), connection_params.getConnectionURL()); client = object_storage->getAzureBlobStorageClient(); - auto settings_copy = *object_storage->getSettings(); - settings_copy.use_native_copy = allow_azure_native_copy; - settings = std::make_unique(settings_copy); + settings = object_storage->getSettings(); } void BackupWriterAzureBlobStorage::copyFileFromDisk( diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ebdb6860986..9d3fedc3063 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -112,16 +112,22 @@ class IColumn; M(Bool, s3_use_adaptive_timeouts, S3::DEFAULT_USE_ADAPTIVE_TIMEOUTS, "When adaptive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ + M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ + M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ + M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \ M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ + M(UInt64, azure_sdk_max_retries, 10, "Maximum number of retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff beetween retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff beetween retries in azure sdk", 0) \ M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, S3::DEFAULT_DISABLE_CHECKSUM, "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, S3::DEFAULT_RETRY_ATTEMPTS, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index c2e4bc0dc89..d9dfedadd48 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -197,7 +197,7 @@ void processURL(const String & url, const String & container_name, Endpoint & en return; } - size_t pos = url.find('?'); + auto pos = url.find('?'); /// If conneciton_url does not have '?', then its not SAS if (pos == std::string::npos) @@ -273,42 +273,60 @@ BlobClientOptions getClientOptions(const RequestSettings & settings, bool for_di std::unique_ptr getRequestSettings(const Settings & query_settings) { - auto settings_ptr = std::make_unique(); + auto settings = std::make_unique(); - settings_ptr->max_single_part_upload_size = query_settings.azure_max_single_part_upload_size; - settings_ptr->max_single_read_retries = query_settings.azure_max_single_read_retries; - settings_ptr->list_object_keys_size = static_cast(query_settings.azure_list_object_keys_size); + settings->max_single_part_upload_size = query_settings.azure_max_single_part_upload_size; + settings->max_single_read_retries = query_settings.azure_max_single_read_retries; + settings->max_single_download_retries = query_settings.azure_max_single_read_retries; + settings->list_object_keys_size = query_settings.azure_list_object_keys_size; + settings->min_upload_part_size = query_settings.azure_min_upload_part_size; + settings->max_upload_part_size = query_settings.azure_max_upload_part_size; + settings->max_single_part_copy_size = query_settings.azure_max_single_part_copy_size; + settings->max_blocks_in_multipart_upload = query_settings.azure_max_blocks_in_multipart_upload; + settings->max_unexpected_write_error_retries = query_settings.azure_max_unexpected_write_error_retries; + settings->max_inflight_parts_for_one_file = query_settings.azure_max_inflight_parts_for_one_file; + settings->strict_upload_part_size = query_settings.azure_strict_upload_part_size; + settings->upload_part_size_multiply_factor = query_settings.azure_upload_part_size_multiply_factor; + settings->upload_part_size_multiply_parts_count_threshold = query_settings.azure_upload_part_size_multiply_parts_count_threshold; + settings->sdk_max_retries = query_settings.azure_sdk_max_retries; + settings->sdk_retry_initial_backoff_ms = query_settings.azure_sdk_retry_initial_backoff_ms; + settings->sdk_retry_max_backoff_ms = query_settings.azure_sdk_retry_max_backoff_ms; - settings_ptr->sdk_max_retries = query_settings.azure_sdk_max_retries; - settings_ptr->sdk_retry_initial_backoff_ms = query_settings.azure_sdk_retry_initial_backoff_ms; - settings_ptr->sdk_retry_max_backoff_ms = query_settings.azure_sdk_retry_max_backoff_ms; + return settings; +} - return settings_ptr; +std::unique_ptr getRequestSettingsForBackup(const Settings & query_settings, bool use_native_copy) +{ + auto settings = getRequestSettings(query_settings); + settings->use_native_copy = use_native_copy; + return settings; } std::unique_ptr getRequestSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { auto settings = std::make_unique(); + const auto & settings_ref = context->getSettingsRef(); - settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", context->getSettings().azure_max_single_part_upload_size); settings->min_bytes_for_seek = config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024); - settings->max_single_read_retries = config.getInt(config_prefix + ".max_single_read_retries", 3); - settings->max_single_download_retries = config.getInt(config_prefix + ".max_single_download_retries", 3); - settings->list_object_keys_size = config.getInt(config_prefix + ".list_object_keys_size", 1000); - settings->min_upload_part_size = config.getUInt64(config_prefix + ".min_upload_part_size", context->getSettings().azure_min_upload_part_size); - settings->max_upload_part_size = config.getUInt64(config_prefix + ".max_upload_part_size", context->getSettings().azure_max_upload_part_size); - settings->max_single_part_copy_size = config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size); settings->use_native_copy = config.getBool(config_prefix + ".use_native_copy", false); - settings->max_blocks_in_multipart_upload = config.getUInt64(config_prefix + ".max_blocks_in_multipart_upload", 50000); - settings->max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries); - settings->max_inflight_parts_for_one_file = config.getUInt64(config_prefix + ".max_inflight_parts_for_one_file", context->getSettings().azure_max_inflight_parts_for_one_file); - settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); - settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); - settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); - settings->sdk_max_retries = config.getUInt(config_prefix + ".max_tries", 10); - settings->sdk_retry_initial_backoff_ms = config.getUInt(config_prefix + ".retry_initial_backoff_ms", 10); - settings->sdk_retry_max_backoff_ms = config.getUInt(config_prefix + ".retry_max_backoff_ms", 1000); + settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", settings_ref.azure_max_single_part_upload_size); + settings->max_single_read_retries = config.getUInt64(config_prefix + ".max_single_read_retries", settings_ref.azure_max_single_read_retries); + settings->max_single_download_retries = config.getUInt64(config_prefix + ".max_single_download_retries", settings_ref.azure_max_single_read_retries); + settings->list_object_keys_size = config.getUInt64(config_prefix + ".list_object_keys_size", settings_ref.azure_list_object_keys_size); + settings->min_upload_part_size = config.getUInt64(config_prefix + ".min_upload_part_size", settings_ref.azure_min_upload_part_size); + settings->max_upload_part_size = config.getUInt64(config_prefix + ".max_upload_part_size", settings_ref.azure_max_upload_part_size); + settings->max_single_part_copy_size = config.getUInt64(config_prefix + ".max_single_part_copy_size", settings_ref.azure_max_single_part_copy_size); + settings->max_blocks_in_multipart_upload = config.getUInt64(config_prefix + ".max_blocks_in_multipart_upload", settings_ref.azure_max_blocks_in_multipart_upload); + settings->max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", settings_ref.azure_max_unexpected_write_error_retries); + settings->max_inflight_parts_for_one_file = config.getUInt64(config_prefix + ".max_inflight_parts_for_one_file", settings_ref.azure_max_inflight_parts_for_one_file); + settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", settings_ref.azure_strict_upload_part_size); + settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", settings_ref.azure_upload_part_size_multiply_factor); + settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", settings_ref.azure_upload_part_size_multiply_parts_count_threshold); + + settings->sdk_max_retries = config.getUInt64(config_prefix + ".max_tries", settings_ref.azure_sdk_max_retries); + settings->sdk_retry_initial_backoff_ms = config.getUInt64(config_prefix + ".retry_initial_backoff_ms", settings_ref.azure_sdk_retry_initial_backoff_ms); + settings->sdk_retry_max_backoff_ms = config.getUInt64(config_prefix + ".retry_max_backoff_ms", settings_ref.azure_sdk_retry_max_backoff_ms); if (config.has(config_prefix + ".curl_ip_resolve")) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index 5f9f280ad4a..19ba48ea225 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -32,22 +32,23 @@ struct RequestSettings RequestSettings() = default; size_t max_single_part_upload_size = 100 * 1024 * 1024; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset - uint64_t min_bytes_for_seek = 1024 * 1024; + size_t min_bytes_for_seek = 1024 * 1024; size_t max_single_read_retries = 3; size_t max_single_download_retries = 3; - int list_object_keys_size = 1000; + size_t list_object_keys_size = 1000; size_t min_upload_part_size = 16 * 1024 * 1024; size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t max_single_part_copy_size = 256 * 1024 * 1024; - bool use_native_copy = false; size_t max_unexpected_write_error_retries = 4; size_t max_inflight_parts_for_one_file = 20; + size_t max_blocks_in_multipart_upload = 50000; size_t strict_upload_part_size = 0; size_t upload_part_size_multiply_factor = 2; size_t upload_part_size_multiply_parts_count_threshold = 500; size_t sdk_max_retries = 10; size_t sdk_retry_initial_backoff_ms = 10; size_t sdk_retry_max_backoff_ms = 1000; + bool use_native_copy = false; using CurlOptions = Azure::Core::Http::CurlTransportOptions; CurlOptions::CurlOptIPResolve curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_WHATEVER; @@ -125,7 +126,9 @@ std::unique_ptr getContainerClient(const ConnectionParams & par BlobClientOptions getClientOptions(const RequestSettings & settings, bool for_disk); AuthMethod getAuthMethod(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); + std::unique_ptr getRequestSettings(const Settings & query_settings); +std::unique_ptr getRequestSettingsForBackup(const Settings & query_settings, bool use_native_copy); std::unique_ptr getRequestSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 3f32b4b410e..092277aca50 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -332,8 +332,9 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) ObjectStorageType::Azure, config, config_prefix, name, AzureBlobStorage::getContainerClient(params, /*readonly=*/ false), std::move(azure_settings), params.endpoint.prefix.empty() ? params.endpoint.container_name : params.endpoint.container_name + "/" + params.endpoint.prefix, - endpoint.getEndpointWithoutContainer()); + params.endpoint.getEndpointWithoutContainer()); }; + factory.registerObjectStorageType("azure_blob_storage", creator); factory.registerObjectStorageType("azure", creator); } diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index e4b3d61f659..595d4da3609 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -73,10 +73,16 @@ StorageObjectStorage::QuerySettings StorageAzureConfiguration::getQuerySettings( ObjectStoragePtr StorageAzureConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { assertInitialized(); - auto client = createClient(is_readonly, /* attempt_to_create_container */true); - auto settings = createSettings(context); + + auto settings = AzureBlobStorage::getRequestSettings(context->getSettingsRef()); + auto client = AzureBlobStorage::getContainerClient(connection_params, is_readonly); + return std::make_unique( - "AzureBlobStorage", std::move(client), std::move(settings), container, getConnectionURL().toString()); + "AzureBlobStorage", + connection_params.createForContainer(), + std::move(settings), + connection_params.getContainer(), + connection_params.getConnectionURL()); } static AzureBlobStorage::ConnectionParams getConnectionParams( From 51f300356e0c0df9c7d001ffe0b7967c7d9f438e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jun 2024 23:57:33 +0000 Subject: [PATCH 028/107] fix style --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d3fedc3063..067e46226ea 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -126,8 +126,8 @@ class IColumn; M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ M(UInt64, azure_sdk_max_retries, 10, "Maximum number of retries in azure sdk", 0) \ - M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff beetween retries in azure sdk", 0) \ - M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff beetween retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_initial_backoff_ms, 10, "Minimal backoff between retries in azure sdk", 0) \ + M(UInt64, azure_sdk_retry_max_backoff_ms, 1000, "Maximal backoff between retries in azure sdk", 0) \ M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, S3::DEFAULT_DISABLE_CHECKSUM, "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, S3::DEFAULT_RETRY_ATTEMPTS, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ From 615cd96c6e6893973b14cc0190e510894b747b22 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 25 Jun 2024 12:48:43 +0100 Subject: [PATCH 029/107] fix test --- tests/integration/test_memory_limit_observer/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_memory_limit_observer/test.py b/tests/integration/test_memory_limit_observer/test.py index f19e119c019..2840c830396 100644 --- a/tests/integration/test_memory_limit_observer/test.py +++ b/tests/integration/test_memory_limit_observer/test.py @@ -55,9 +55,9 @@ def test_observe_memory_limit(started_cluster): def test_memory_usage_doesnt_include_page_cache_size(started_cluster): try: - # populate page cache with 10GB of data; it might be killed by OOM killer but it is fine + # populate page cache with 4GB of data; it might be killed by OOM killer but it is fine node1.exec_in_container( - ["dd", "if=/dev/zero", "of=outputfile", "bs=1M", "count=10K"] + ["dd", "if=/dev/zero", "of=outputfile", "bs=1M", "count=4K"] ) except Exception: pass @@ -76,4 +76,4 @@ def test_memory_usage_doesnt_include_page_cache_size(started_cluster): WHERE logger_name = 'CgroupsMemoryUsageObserver' AND message LIKE 'Read current memory usage%bytes%' """ ).strip() - assert int(max_mem_usage_from_cgroup) < 2 * 2**30 + assert int(max_mem_usage_from_cgroup) < 2 * 2 ** 30 From d9f681b39d22a99f8ace10f39ed308621b27a774 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 26 Jun 2024 10:44:17 +0200 Subject: [PATCH 030/107] Disable stacktrace collection in GWPAsan by default --- src/Common/GWPAsan.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 488f8e2c5dc..ea376609ff4 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,9 +57,12 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 50000; + opts.SampleRate = 5000; + + const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) + if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") + opts.Backtrace = getBackTrace; - opts.Backtrace = getBackTrace; GuardedAlloc.init(opts); return true; From c30ecee10c8cba5d245916d458ddd60345ea359c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 26 Jun 2024 12:40:55 +0000 Subject: [PATCH 031/107] remove unused code --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 +- src/Storages/ObjectStorage/Azure/Configuration.h | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index baad3bdf223..2c7ce5e18dc 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -1,5 +1,4 @@ #pragma once -#include "Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h" #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -9,6 +8,7 @@ #include #include #include +#include namespace Poco { diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 272d155e337..4e6bfbc0745 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -57,9 +57,6 @@ protected: std::string blob_path; std::vector blobs_paths; AzureBlobStorage::ConnectionParams connection_params; - - // AzureClientPtr createClient(bool is_read_only, bool attempt_to_create_container); - // AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); }; } From 5ddb9b11f487b3f13cb6d7e1d69e22779b6a745f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 27 Jun 2024 14:33:37 +0000 Subject: [PATCH 032/107] remove unwanted changes --- src/Backups/BackupIO_AzureBlobStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 0ee0160a969..cee41861d70 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -36,7 +36,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getContainer(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false} , connection_params(connection_params_) , blob_path(blob_path_) { @@ -128,7 +128,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const ContextPtr & context_, bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getContainer(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false} , connection_params(connection_params_) , blob_path(blob_path_) { From cb3d0ed2757fc6de98fdb0bad1e74f83facd7c88 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Fri, 28 Jun 2024 02:20:35 +0200 Subject: [PATCH 033/107] Update StorageMaterializedView.cpp --- src/Storages/StorageMaterializedView.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 316f398b476..f9f627863dd 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -161,6 +161,7 @@ StorageMaterializedView::StorageMaterializedView( manual_create_query->setDatabase(getStorageID().database_name); manual_create_query->setTable(generateInnerTableName(getStorageID())); manual_create_query->uuid = query.to_inner_uuid; + manual_create_query->has_uuid = true; auto new_columns_list = std::make_shared(); new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr()); From b16451ad8946fdeb93ca259af083467853b6ac22 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Fri, 28 Jun 2024 02:28:07 +0200 Subject: [PATCH 034/107] Update StorageMaterializedView.cpp --- src/Storages/StorageMaterializedView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index f9f627863dd..ec1559b71a4 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -161,7 +161,7 @@ StorageMaterializedView::StorageMaterializedView( manual_create_query->setDatabase(getStorageID().database_name); manual_create_query->setTable(generateInnerTableName(getStorageID())); manual_create_query->uuid = query.to_inner_uuid; - manual_create_query->has_uuid = true; + manual_create_query->has_uuid = query.to_inner_uuid != UUIDHelpers::Nil; auto new_columns_list = std::make_shared(); new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr()); From fa7bad4993dee91009cc275fb49755aee0bf849d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jun 2024 13:51:42 +0200 Subject: [PATCH 035/107] Decrease sampling rate slightly --- src/Common/GWPAsan.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index ea376609ff4..f4a916a696b 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 5000; + opts.SampleRate = 6000; const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") From 31c65a40926d3d5209898f5efb5c1cf33b602133 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Jun 2024 14:21:14 +0000 Subject: [PATCH 036/107] add settings to change history --- src/Core/SettingsChangesHistory.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4ac25a649b7..1c5ad9d0875 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -87,6 +87,9 @@ namespace SettingsChangesHistory static const std::map settings_changes_history = { {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, + {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, + {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, From 898dd8bb8efd260733c8b967868e4fcb88fb145e Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 28 Jun 2024 15:15:19 +0000 Subject: [PATCH 037/107] Throw exception in bitShift for negative shift positions --- src/Functions/bitShiftLeft.cpp | 6 ++++++ src/Functions/bitShiftRight.cpp | 7 +++++++ ...ift_throws_error_for_negative_shift_positions.reference | 0 ...bit_shift_throws_error_for_negative_shift_positions.sql | 7 +++++++ 4 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference create mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index c366a1ecb44..c3f5de628aa 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -24,6 +24,8 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else if constexpr (is_big_int_v) return static_cast(a) << static_cast(b); else @@ -35,6 +37,8 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; @@ -100,6 +104,8 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 1c37cd3bf4c..b53485c45f5 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -8,6 +8,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -25,6 +26,8 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else if constexpr (is_big_int_v) return static_cast(a) >> static_cast(b); else @@ -51,6 +54,8 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; @@ -88,6 +93,8 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); + else if (b < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql new file mode 100644 index 00000000000..659d03d1951 --- /dev/null +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql @@ -0,0 +1,7 @@ +SELECT bitShiftRight(1, -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT bitShiftRight('hola', -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT bitShiftRight(toFixedString('hola', 10), -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT bitShiftLeft(1, -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT bitShiftLeft('hola', -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT bitShiftLeft(toFixedString('hola', 10), -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } \ No newline at end of file From 77c8f034597639439e7b9c09ea89207c73cd398e Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 28 Jun 2024 17:26:03 +0000 Subject: [PATCH 038/107] Fix coding style --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 380656cd1ca..31972894c3d 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -94,6 +94,7 @@ EXTERN_TYPES_EXCLUDES=( ErrorCodes::values[i] ErrorCodes::getErrorCodeByName ErrorCodes::Value + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT ) for extern_type in ${!EXTERN_TYPES[@]}; do type_of_extern=${EXTERN_TYPES[$extern_type]} From 15d9ad65c65a476e8573ec37aca25050a8a8f7a4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 1 Jul 2024 09:01:32 +0200 Subject: [PATCH 039/107] Reduce even more --- src/Common/GWPAsan.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index f4a916a696b..0482ddb4e2b 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 6000; + opts.SampleRate = 8000; const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") From 0b24a416b5f50bb4416d4d95dddd20a1e333b569 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 1 Jul 2024 10:31:29 +0000 Subject: [PATCH 040/107] Also throw error if bit shift positions is greater than the bit width of value --- src/Functions/bitShiftLeft.cpp | 19 ++++++++++------- src/Functions/bitShiftRight.cpp | 19 +++++++++-------- ...ror_for_negative_shift_positions.reference | 0 ...ows_error_for_negative_shift_positions.sql | 7 ------- ...t_throws_error_for_out_of_bounds.reference | 1 + ...t_shift_throws_error_for_out_of_bounds.sql | 21 +++++++++++++++++++ 6 files changed, 43 insertions(+), 24 deletions(-) delete mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference delete mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql create mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference create mode 100644 tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index c3f5de628aa..9d32e5b5ca4 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -7,6 +7,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace @@ -24,8 +25,8 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); + else if (b < 0 || b > B(8 * sizeof(A))) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) << static_cast(b); else @@ -37,13 +38,15 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; + size_t n = end - pos; + if (b < 0 || b > B(word_size * n)) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + /// To prevent overflow - if (static_cast(b) >= (static_cast(end - pos) * word_size) || b < 0) + if (static_cast(b) >= (static_cast(n) * word_size)) { // insert default value out_vec.push_back(0); @@ -104,14 +107,14 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; size_t n = end - pos; + if (b < 0 || b > B(word_size * n)) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size) || b < 0) + if (static_cast(b) >= (static_cast(n) * word_size)) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index b53485c45f5..13b210a4f63 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -8,7 +8,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace @@ -26,8 +26,8 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); + else if (b < 0 || b > B(8 * sizeof(A))) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) >> static_cast(b); else @@ -54,13 +54,14 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; + size_t n = end - pos; + if (b < 0 || b > B(word_size * n)) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); /// To prevent overflow - if (static_cast(b) >= (static_cast(end - pos) * word_size) || b < 0) + if (static_cast(b) >= (static_cast(n) * word_size)) { /// insert default value out_vec.push_back(0); @@ -93,14 +94,14 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); - else if (b < 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The number of shift positions needs to be a positive value"); else { UInt8 word_size = 8; size_t n = end - pos; + if (b < 0 || b > B(word_size * n)) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size) || b < 0) + if (static_cast(b) >= (static_cast(n) * word_size)) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql deleted file mode 100644 index 659d03d1951..00000000000 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_negative_shift_positions.sql +++ /dev/null @@ -1,7 +0,0 @@ -SELECT bitShiftRight(1, -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT bitShiftRight('hola', -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT bitShiftRight(toFixedString('hola', 10), -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } - -SELECT bitShiftLeft(1, -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT bitShiftLeft('hola', -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT bitShiftLeft(toFixedString('hola', 10), -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } \ No newline at end of file diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql new file mode 100644 index 00000000000..9cfc6f00b91 --- /dev/null +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql @@ -0,0 +1,21 @@ +SELECT bitShiftRight(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight(toUInt8(1), number) FROM numbers(8 + 1) FORMAT Null; +SELECT bitShiftRight(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight('hola', number) FROM numbers(4 * 8 + 1) FORMAT Null; +SELECT bitShiftRight('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight(toFixedString('hola', 8), number) FROM numbers(8 * 8 + 1) FORMAT Null; +SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } + +SELECT bitShiftLeft(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft(toUInt8(1), number) FROM numbers(8 + 1) FORMAT Null; +SELECT bitShiftLeft(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft('hola', number) FROM numbers(4 * 8 + 1) FORMAT Null; +SELECT bitShiftLeft('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft(toFixedString('hola', 8), number) FROM numbers(8 * 8 + 1) FORMAT Null; +SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } + +SELECT 'OK'; \ No newline at end of file From 0fed338ac26bb7a974fe8cfe8ac1b6dacbc3f4df Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 1 Jul 2024 11:03:25 +0000 Subject: [PATCH 041/107] Fix other bit shift tests after out of bounds check --- ...t_shift_right_for_string_integer.reference | 39 ------------------- ...016_bit_shift_right_for_string_integer.sql | 19 ++++----- ...it_shift_left_for_string_integer.reference | 39 ------------------- ...2017_bit_shift_left_for_string_integer.sql | 19 ++++----- ...t_shift_throws_error_for_out_of_bounds.sql | 6 --- 5 files changed, 16 insertions(+), 106 deletions(-) diff --git a/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.reference b/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.reference index e6a2b2b6aaf..ab832478da0 100644 --- a/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.reference +++ b/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.reference @@ -41,8 +41,6 @@ String ConstConst 38 Hello 00000001 39 Hello 00000000 40 Hello -41 Hello -42 Hello FixedString ConstConst 1 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 @@ -92,10 +90,8 @@ FixedString ConstConst 78 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000001 79 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 80 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -81 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 String VectorVector --1 Hello 0 Hello 0100100001100101011011000110110001101111 1 Hello 0010010000110010101101100011011000110111 7 Hello 0000000010010000110010101101100011011000 @@ -112,8 +108,6 @@ String VectorVector 33 Hello 00100100 39 Hello 00000000 40 Hello -41 Hello -42 Hello 7 Hel 000000001001000011001010 8 Hel 0100100001100101 9 Hel 0010010000110010 @@ -125,7 +119,6 @@ String VectorVector 9 Hel 0010010000110010 FixedString VectorVector --1 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 1 Hello\0\0\0\0\0 00100100001100101011011000110110001101111000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 @@ -142,8 +135,6 @@ FixedString VectorVector 33 Hello\0\0\0\0\0 00000000000000000000000000000000001001000011001010110110001101100011011110000000 39 Hello\0\0\0\0\0 00000000000000000000000000000000000000001001000011001010110110001101100011011110 40 Hello\0\0\0\0\0 00000000000000000000000000000000000000000100100001100101011011000110110001101111 -41 Hello\0\0\0\0\0 00000000000000000000000000000000000000000010010000110010101101100011011000110111 -42 Hello\0\0\0\0\0 00000000000000000000000000000000000000000001001000011001010110110001101100011011 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 00000000010010000110010101101100000000000000000000000000000000000000000000000000 9 Hel\0\0\0\0\0\0\0 00000000001001000011001010110110000000000000000000000000000000000000000000000000 @@ -171,9 +162,6 @@ String VectorConst 7 Hello 0000000010010000110010101101100011011000 7 Hello 0000000010010000110010101101100011011000 7 Hello 0000000010010000110010101101100011011000 -7 Hello 0000000010010000110010101101100011011000 -7 Hello 0000000010010000110010101101100011011000 -7 Hello 0000000010010000110010101101100011011000 7 Hel 000000001001000011001010 7 Hel 000000001001000011001010 7 Hel 000000001001000011001010 @@ -193,9 +181,6 @@ String VectorConst 8 Hello 01001000011001010110110001101100 8 Hello 01001000011001010110110001101100 8 Hello 01001000011001010110110001101100 -8 Hello 01001000011001010110110001101100 -8 Hello 01001000011001010110110001101100 -8 Hello 01001000011001010110110001101100 8 Hel 0100100001100101 8 Hel 0100100001100101 8 Hel 0100100001100101 @@ -217,9 +202,6 @@ FixedString VectorConst 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 @@ -239,15 +221,11 @@ FixedString VectorConst 8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 -8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 -8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 -8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 00000000010010000110010101101100000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 00000000010010000110010101101100000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 00000000010010000110010101101100000000000000000000000000000000000000000000000000 String ConstVector --1 Hello 0 Hello 0100100001100101011011000110110001101111 1 Hello 0010010000110010101101100011011000110111 7 Hello 0000000010010000110010101101100011011000 @@ -264,12 +242,9 @@ String ConstVector 33 Hello 00100100 39 Hello 00000000 40 Hello -41 Hello -42 Hello 7 Hello 0000000010010000110010101101100011011000 8 Hello 01001000011001010110110001101100 9 Hello 00100100001100101011011000110110 --1 Hel 0 Hel 010010000110010101101100 1 Hel 001001000011001010110110 7 Hel 000000001001000011001010 @@ -280,20 +255,11 @@ String ConstVector 17 Hel 00100100 23 Hel 00000000 24 Hel -25 Hel -31 Hel -32 Hel -33 Hel -39 Hel -40 Hel -41 Hel -42 Hel 7 Hel 000000001001000011001010 8 Hel 0100100001100101 9 Hel 0010010000110010 FixedString ConstVector --1 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 1 Hello\0\0\0\0\0 00100100001100101011011000110110001101111000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 @@ -310,12 +276,9 @@ FixedString ConstVector 33 Hello\0\0\0\0\0 00000000000000000000000000000000001001000011001010110110001101100011011110000000 39 Hello\0\0\0\0\0 00000000000000000000000000000000000000001001000011001010110110001101100011011110 40 Hello\0\0\0\0\0 00000000000000000000000000000000000000000100100001100101011011000110110001101111 -41 Hello\0\0\0\0\0 00000000000000000000000000000000000000000010010000110010101101100011011000110111 -42 Hello\0\0\0\0\0 00000000000000000000000000000000000000000001001000011001010110110001101100011011 7 Hello\0\0\0\0\0 00000000100100001100101011011000110110001101111000000000000000000000000000000000 8 Hello\0\0\0\0\0 00000000010010000110010101101100011011000110111100000000000000000000000000000000 9 Hello\0\0\0\0\0 00000000001001000011001010110110001101100011011110000000000000000000000000000000 --1 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hel\0\0\0\0\0\0\0 01001000011001010110110000000000000000000000000000000000000000000000000000000000 1 Hel\0\0\0\0\0\0\0 00100100001100101011011000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 @@ -332,8 +295,6 @@ FixedString ConstVector 33 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000001001000011001010110110000000000000000000000000 39 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000001001000011001010110110000000000000000000 40 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000100100001100101011011000000000000000000 -41 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000010010000110010101101100000000000000000 -42 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000001001000011001010110110000000000000000 7 Hel\0\0\0\0\0\0\0 00000000100100001100101011011000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 00000000010010000110010101101100000000000000000000000000000000000000000000000000 9 Hel\0\0\0\0\0\0\0 00000000001001000011001010110110000000000000000000000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.sql b/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.sql index 0ee04e408ba..40fccbc89e6 100644 --- a/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.sql +++ b/tests/queries/0_stateless/02016_bit_shift_right_for_string_integer.sql @@ -41,8 +41,6 @@ SELECT 37,'Hello',bin(bitShiftRight('Hello', 37)); SELECT 38,'Hello',bin(bitShiftRight('Hello', 38)); SELECT 39,'Hello',bin(bitShiftRight('Hello', 39)); SELECT 40,'Hello',bin(bitShiftRight('Hello', 40)); -SELECT 41,'Hello',bin(bitShiftRight('Hello', 41)); -SELECT 42,'Hello',bin(bitShiftRight('Hello', 42)); SELECT 'FixedString ConstConst'; SELECT bin(toFixedString('Hello', 10)) == bin(bitShiftRight(toFixedString('Hello', 10), 0)); @@ -93,40 +91,39 @@ SELECT 77,toFixedString('Hello', 10), bin(bitShiftRight(toFixedString('Hello', 1 SELECT 78,toFixedString('Hello', 10), bin(bitShiftRight(toFixedString('Hello', 10), 78)); SELECT 79,toFixedString('Hello', 10), bin(bitShiftRight(toFixedString('Hello', 10), 79)); SELECT 80,toFixedString('Hello', 10), bin(bitShiftRight(toFixedString('Hello', 10), 80)); -SELECT 81,toFixedString('Hello', 10), bin(bitShiftRight(toFixedString('Hello', 10), 81)); DROP TABLE IF EXISTS test_bit_shift_right_string_integer; CREATE TABLE test_bit_shift_right_string_integer (str String, fixedStr FixedString(10), id Int64) engine=Log; -INSERT INTO test_bit_shift_right_string_integer VALUES('Hello','Hello',-1)('Hello','Hello',0),('Hello','Hello',1),('Hello','Hello',7),('Hello','Hello',8),('Hello','Hello',9),('Hello','Hello',15),('Hello','Hello',16),('Hello','Hello',17),('Hello','Hello',23),('Hello','Hello',24),('Hello','Hello',25),('Hello','Hello',31),('Hello','Hello',32),('Hello','Hello',33),('Hello','Hello',39),('Hello','Hello',40),('Hello','Hello',41),('Hello','Hello',42),('Hel','Hel',7),('Hel','Hel',8),('Hel','Hel',9); +INSERT INTO test_bit_shift_right_string_integer VALUES('Hello','Hello',0),('Hello','Hello',1),('Hello','Hello',7),('Hello','Hello',8),('Hello','Hello',9),('Hello','Hello',15),('Hello','Hello',16),('Hello','Hello',17),('Hello','Hello',23),('Hello','Hello',24),('Hello','Hello',25),('Hello','Hello',31),('Hello','Hello',32),('Hello','Hello',33),('Hello','Hello',39),('Hello','Hello',40),('Hel','Hel',7),('Hel','Hel',8),('Hel','Hel',9); -SELECT bin(bitShiftRight('Hello', 42)); --A blank line +SELECT bin(bitShiftRight('Hello', 40)); --A blank line SELECT 'String VectorVector'; SELECT id as shift_right_bit,str as arg,bin(bitShiftRight(str, id)) as string_res FROM test_bit_shift_right_string_integer; SELECT id as shift_right_bit,str as arg,bin(bitShiftRight(str, id)) as string_res FROM test_bit_shift_right_string_integer WHERE (str='Hello' AND (id=23 OR id=24 OR id=25)) OR (str='Hel' AND (id=7 OR id=8 OR id=9)); -SELECT bin(bitShiftRight('Hello', 42)); +SELECT bin(bitShiftRight('Hello', 40)); SELECT 'FixedString VectorVector'; SELECT id as shift_right_bit,fixedStr as arg,bin(bitShiftRight(fixedStr, id)) as fixed_string_res FROM test_bit_shift_right_string_integer; SELECT id as shift_right_bit,fixedStr as arg,bin(bitShiftRight(fixedStr, id)) as fixed_string_res FROM test_bit_shift_right_string_integer WHERE (str='Hello' AND (id=23 OR id=24 OR id=25)) OR (str='Hel' AND (id=7 OR id=8 OR id=9)); -SELECT bin(bitShiftRight('Hello', 42)); --A blank line +SELECT bin(bitShiftRight('Hello', 40)); --A blank line SELECT 'String VectorConst'; SELECT 7 as shift_right_bit,str as arg,bin(bitShiftRight(str, 7)) as string_res FROM test_bit_shift_right_string_integer; SELECT 8 as shift_right_bit,str as arg,bin(bitShiftRight(str, 8)) as string_res FROM test_bit_shift_right_string_integer; -SELECT bin(bitShiftRight('Hello', 42)); --A blank line +SELECT bin(bitShiftRight('Hello', 40)); --A blank line SELECT 'FixedString VectorConst'; SELECT 7 as shift_right_bit,fixedStr as arg,bin(bitShiftRight(fixedStr, 7)) as fixed_string_res FROM test_bit_shift_right_string_integer; SELECT 8 as shift_right_bit,fixedStr as arg,bin(bitShiftRight(fixedStr, 8)) as fixed_string_res FROM test_bit_shift_right_string_integer; -SELECT bin(bitShiftRight('Hello', 42)); --A blank line +SELECT bin(bitShiftRight('Hello', 40)); --A blank line SELECT 'String ConstVector'; SELECT id as shift_right_bit,'Hello' as arg,bin(bitShiftRight('Hello', id)) as string_res FROM test_bit_shift_right_string_integer; -SELECT id as shift_right_bit,'Hel' as arg,bin(bitShiftRight('Hel', id)) as string_res FROM test_bit_shift_right_string_integer; +SELECT id as shift_right_bit,'Hel' as arg,bin(bitShiftRight('Hel', id)) as string_res FROM test_bit_shift_right_string_integer WHERE id <= 8 * 3; -SELECT bin(bitShiftRight('Hello', 42)); --A blank line +SELECT bin(bitShiftRight('Hello', 40)); --A blank line SELECT 'FixedString ConstVector'; SELECT id as shift_right_bit,toFixedString('Hello', 10) as arg,bin(bitShiftRight(toFixedString('Hello', 10), id)) as fixed_string_res FROM test_bit_shift_right_string_integer; SELECT id as shift_right_bit,toFixedString('Hel', 10) as arg,bin(bitShiftRight(toFixedString('Hel', 10), id)) as fixed_string_res FROM test_bit_shift_right_string_integer; diff --git a/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.reference b/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.reference index ff5a09c0d48..a20c44bbb9a 100644 --- a/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.reference +++ b/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.reference @@ -41,8 +41,6 @@ String ConstConst 38 Hello 00010010000110010101101100011011000110111100000000000000000000000000000000000000 39 Hello 00100100001100101011011000110110001101111000000000000000000000000000000000000000 40 Hello -41 Hello -42 Hello FixedString ConstConst 1 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 @@ -92,10 +90,8 @@ FixedString ConstConst 78 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 79 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 80 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -81 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 String VectorVector --1 Hello 0 Hello 0100100001100101011011000110110001101111 1 Hello 000000001001000011001010110110001101100011011110 7 Hello 001001000011001010110110001101100011011110000000 @@ -112,8 +108,6 @@ String VectorVector 33 Hello 00000000100100001100101011011000110110001101111000000000000000000000000000000000 39 Hello 00100100001100101011011000110110001101111000000000000000000000000000000000000000 40 Hello -41 Hello -42 Hello 7 Hel 00100100001100101011011000000000 8 Hel 01001000011001010110110000000000 9 Hel 0000000010010000110010101101100000000000 @@ -125,7 +119,6 @@ String VectorVector 9 Hel 0000000010010000110010101101100000000000 FixedString VectorVector --1 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 1 Hello\0\0\0\0\0 10010000110010101101100011011000110111100000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 @@ -142,8 +135,6 @@ FixedString VectorVector 33 Hello\0\0\0\0\0 11011110000000000000000000000000000000000000000000000000000000000000000000000000 39 Hello\0\0\0\0\0 10000000000000000000000000000000000000000000000000000000000000000000000000000000 40 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -41 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -42 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 01100101011011000000000000000000000000000000000000000000000000000000000000000000 9 Hel\0\0\0\0\0\0\0 11001010110110000000000000000000000000000000000000000000000000000000000000000000 @@ -171,9 +162,6 @@ String VectorConst 7 Hello 001001000011001010110110001101100011011110000000 7 Hello 001001000011001010110110001101100011011110000000 7 Hello 001001000011001010110110001101100011011110000000 -7 Hello 001001000011001010110110001101100011011110000000 -7 Hello 001001000011001010110110001101100011011110000000 -7 Hello 001001000011001010110110001101100011011110000000 7 Hel 00100100001100101011011000000000 7 Hel 00100100001100101011011000000000 7 Hel 00100100001100101011011000000000 @@ -193,9 +181,6 @@ String VectorConst 8 Hello 010010000110010101101100011011000110111100000000 8 Hello 010010000110010101101100011011000110111100000000 8 Hello 010010000110010101101100011011000110111100000000 -8 Hello 010010000110010101101100011011000110111100000000 -8 Hello 010010000110010101101100011011000110111100000000 -8 Hello 010010000110010101101100011011000110111100000000 8 Hel 01001000011001010110110000000000 8 Hel 01001000011001010110110000000000 8 Hel 01001000011001010110110000000000 @@ -217,9 +202,6 @@ FixedString VectorConst 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 -7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 @@ -239,15 +221,11 @@ FixedString VectorConst 8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 -8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 -8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 -8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 01100101011011000000000000000000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 01100101011011000000000000000000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 01100101011011000000000000000000000000000000000000000000000000000000000000000000 String ConstVector --1 Hello 0 Hello 0100100001100101011011000110110001101111 1 Hello 000000001001000011001010110110001101100011011110 7 Hello 001001000011001010110110001101100011011110000000 @@ -264,12 +242,9 @@ String ConstVector 33 Hello 00000000100100001100101011011000110110001101111000000000000000000000000000000000 39 Hello 00100100001100101011011000110110001101111000000000000000000000000000000000000000 40 Hello -41 Hello -42 Hello 7 Hello 001001000011001010110110001101100011011110000000 8 Hello 010010000110010101101100011011000110111100000000 9 Hello 00000000100100001100101011011000110110001101111000000000 --1 Hel 0 Hel 010010000110010101101100 1 Hel 00000000100100001100101011011000 7 Hel 00100100001100101011011000000000 @@ -280,20 +255,11 @@ String ConstVector 17 Hel 000000001001000011001010110110000000000000000000 23 Hel 001001000011001010110110000000000000000000000000 24 Hel -25 Hel -31 Hel -32 Hel -33 Hel -39 Hel -40 Hel -41 Hel -42 Hel 7 Hel 00100100001100101011011000000000 8 Hel 01001000011001010110110000000000 9 Hel 0000000010010000110010101101100000000000 FixedString ConstVector --1 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hello\0\0\0\0\0 01001000011001010110110001101100011011110000000000000000000000000000000000000000 1 Hello\0\0\0\0\0 10010000110010101101100011011000110111100000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 @@ -310,12 +276,9 @@ FixedString ConstVector 33 Hello\0\0\0\0\0 11011110000000000000000000000000000000000000000000000000000000000000000000000000 39 Hello\0\0\0\0\0 10000000000000000000000000000000000000000000000000000000000000000000000000000000 40 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -41 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -42 Hello\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 7 Hello\0\0\0\0\0 00110010101101100011011000110111100000000000000000000000000000000000000000000000 8 Hello\0\0\0\0\0 01100101011011000110110001101111000000000000000000000000000000000000000000000000 9 Hello\0\0\0\0\0 11001010110110001101100011011110000000000000000000000000000000000000000000000000 --1 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 0 Hel\0\0\0\0\0\0\0 01001000011001010110110000000000000000000000000000000000000000000000000000000000 1 Hel\0\0\0\0\0\0\0 10010000110010101101100000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 @@ -332,8 +295,6 @@ FixedString ConstVector 33 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 39 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 40 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -41 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 -42 Hel\0\0\0\0\0\0\0 00000000000000000000000000000000000000000000000000000000000000000000000000000000 7 Hel\0\0\0\0\0\0\0 00110010101101100000000000000000000000000000000000000000000000000000000000000000 8 Hel\0\0\0\0\0\0\0 01100101011011000000000000000000000000000000000000000000000000000000000000000000 9 Hel\0\0\0\0\0\0\0 11001010110110000000000000000000000000000000000000000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.sql b/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.sql index 5c7a9901dae..a8e66eda281 100644 --- a/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.sql +++ b/tests/queries/0_stateless/02017_bit_shift_left_for_string_integer.sql @@ -41,8 +41,6 @@ SELECT 37,'Hello',bin(bitShiftLeft('Hello', 37)); SELECT 38,'Hello',bin(bitShiftLeft('Hello', 38)); SELECT 39,'Hello',bin(bitShiftLeft('Hello', 39)); SELECT 40,'Hello',bin(bitShiftLeft('Hello', 40)); -SELECT 41,'Hello',bin(bitShiftLeft('Hello', 41)); -SELECT 42,'Hello',bin(bitShiftLeft('Hello', 42)); SELECT 'FixedString ConstConst'; SELECT bin(toFixedString('Hello', 10)) == bin(bitShiftLeft(toFixedString('Hello', 10), 0)); @@ -93,40 +91,39 @@ SELECT 77,toFixedString('Hello', 10), bin(bitShiftLeft(toFixedString('Hello', 10 SELECT 78,toFixedString('Hello', 10), bin(bitShiftLeft(toFixedString('Hello', 10), 78)); SELECT 79,toFixedString('Hello', 10), bin(bitShiftLeft(toFixedString('Hello', 10), 79)); SELECT 80,toFixedString('Hello', 10), bin(bitShiftLeft(toFixedString('Hello', 10), 80)); -SELECT 81,toFixedString('Hello', 10), bin(bitShiftLeft(toFixedString('Hello', 10), 81)); DROP TABLE IF EXISTS test_bit_shift_left_string_integer; CREATE TABLE test_bit_shift_left_string_integer (str String, fixedStr FixedString(10), id Int64) engine=Log; -INSERT INTO test_bit_shift_left_string_integer VALUES('Hello','Hello',-1)('Hello','Hello',0),('Hello','Hello',1),('Hello','Hello',7),('Hello','Hello',8),('Hello','Hello',9),('Hello','Hello',15),('Hello','Hello',16),('Hello','Hello',17),('Hello','Hello',23),('Hello','Hello',24),('Hello','Hello',25),('Hello','Hello',31),('Hello','Hello',32),('Hello','Hello',33),('Hello','Hello',39),('Hello','Hello',40),('Hello','Hello',41),('Hello','Hello',42),('Hel','Hel',7),('Hel','Hel',8),('Hel','Hel',9); +INSERT INTO test_bit_shift_left_string_integer VALUES('Hello','Hello',0),('Hello','Hello',1),('Hello','Hello',7),('Hello','Hello',8),('Hello','Hello',9),('Hello','Hello',15),('Hello','Hello',16),('Hello','Hello',17),('Hello','Hello',23),('Hello','Hello',24),('Hello','Hello',25),('Hello','Hello',31),('Hello','Hello',32),('Hello','Hello',33),('Hello','Hello',39),('Hello','Hello',40),('Hel','Hel',7),('Hel','Hel',8),('Hel','Hel',9); -SELECT bin(bitShiftLeft('Hello', 42)); --A blank line +SELECT bin(bitShiftLeft('Hello', 40)); --A blank line SELECT 'String VectorVector'; SELECT id as shift_right_bit,str as arg,bin(bitShiftLeft(str, id)) as string_res FROM test_bit_shift_left_string_integer; SELECT id as shift_right_bit,str as arg,bin(bitShiftLeft(str, id)) as string_res FROM test_bit_shift_left_string_integer WHERE (str='Hello' AND (id=23 OR id=24 OR id=25)) OR (str='Hel' AND (id=7 OR id=8 OR id=9)); -SELECT bin(bitShiftLeft('Hello', 42)); +SELECT bin(bitShiftLeft('Hello', 40)); SELECT 'FixedString VectorVector'; SELECT id as shift_right_bit,fixedStr as arg,bin(bitShiftLeft(fixedStr, id)) as fixed_string_res FROM test_bit_shift_left_string_integer; SELECT id as shift_right_bit,fixedStr as arg,bin(bitShiftLeft(fixedStr, id)) as fixed_string_res FROM test_bit_shift_left_string_integer WHERE (str='Hello' AND (id=23 OR id=24 OR id=25)) OR (str='Hel' AND (id=7 OR id=8 OR id=9)); -SELECT bin(bitShiftLeft('Hello', 42)); --A blank line +SELECT bin(bitShiftLeft('Hello', 40)); --A blank line SELECT 'String VectorConst'; SELECT 7 as shift_right_bit,str as arg,bin(bitShiftLeft(str, 7)) as string_res FROM test_bit_shift_left_string_integer; SELECT 8 as shift_right_bit,str as arg,bin(bitShiftLeft(str, 8)) as string_res FROM test_bit_shift_left_string_integer; -SELECT bin(bitShiftLeft('Hello', 42)); --A blank line +SELECT bin(bitShiftLeft('Hello', 40)); --A blank line SELECT 'FixedString VectorConst'; SELECT 7 as shift_right_bit,fixedStr as arg,bin(bitShiftLeft(fixedStr, 7)) as fixed_string_res FROM test_bit_shift_left_string_integer; SELECT 8 as shift_right_bit,fixedStr as arg,bin(bitShiftLeft(fixedStr, 8)) as fixed_string_res FROM test_bit_shift_left_string_integer; -SELECT bin(bitShiftLeft('Hello', 42)); --A blank line +SELECT bin(bitShiftLeft('Hello', 40)); --A blank line SELECT 'String ConstVector'; SELECT id as shift_right_bit,'Hello' as arg,bin(bitShiftLeft('Hello', id)) as string_res FROM test_bit_shift_left_string_integer; -SELECT id as shift_right_bit,'Hel' as arg,bin(bitShiftLeft('Hel', id)) as string_res FROM test_bit_shift_left_string_integer; +SELECT id as shift_right_bit,'Hel' as arg,bin(bitShiftLeft('Hel', id)) as string_res FROM test_bit_shift_left_string_integer WHERE id <= 8 * 3; -SELECT bin(bitShiftLeft('Hello', 42)); --A blank line +SELECT bin(bitShiftLeft('Hello', 40)); --A blank line SELECT 'FixedString ConstVector'; SELECT id as shift_right_bit,toFixedString('Hello', 10) as arg,bin(bitShiftLeft(toFixedString('Hello', 10), id)) as fixed_string_res FROM test_bit_shift_left_string_integer; SELECT id as shift_right_bit,toFixedString('Hel', 10) as arg,bin(bitShiftLeft(toFixedString('Hel', 10), id)) as fixed_string_res FROM test_bit_shift_left_string_integer; diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql index 9cfc6f00b91..a1a246593d8 100644 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql @@ -1,21 +1,15 @@ SELECT bitShiftRight(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight(toUInt8(1), number) FROM numbers(8 + 1) FORMAT Null; SELECT bitShiftRight(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftRight('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight('hola', number) FROM numbers(4 * 8 + 1) FORMAT Null; SELECT bitShiftRight('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftRight(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight(toFixedString('hola', 8), number) FROM numbers(8 * 8 + 1) FORMAT Null; SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftLeft(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft(toUInt8(1), number) FROM numbers(8 + 1) FORMAT Null; SELECT bitShiftLeft(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftLeft('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft('hola', number) FROM numbers(4 * 8 + 1) FORMAT Null; SELECT bitShiftLeft('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftLeft(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft(toFixedString('hola', 8), number) FROM numbers(8 * 8 + 1) FORMAT Null; SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT 'OK'; \ No newline at end of file From d0506f0214e949426c41fea2d9ebd79813422fd8 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 1 Jul 2024 12:23:54 +0000 Subject: [PATCH 042/107] Fix more tests One of tests actually uncovered a casting error :) --- src/Functions/bitShiftLeft.cpp | 2 +- src/Functions/bitShiftRight.cpp | 2 +- tests/queries/0_stateless/02366_kql_func_binary.reference | 3 --- tests/queries/0_stateless/02366_kql_func_binary.sql | 3 --- .../0_stateless/02766_bitshift_with_const_arguments.sql | 2 +- utils/check-style/check-style | 1 - 6 files changed, 3 insertions(+), 10 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 9d32e5b5ca4..3d496296ba9 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -25,7 +25,7 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); - else if (b < 0 || b > B(8 * sizeof(A))) + else if (b < 0 || static_cast(b) > 8 * sizeof(A)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) << static_cast(b); diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 13b210a4f63..0b41493fc6d 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -26,7 +26,7 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); - else if (b < 0 || b > B(8 * sizeof(A))) + else if (b < 0 || static_cast(b) > 8 * sizeof(A)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) >> static_cast(b); diff --git a/tests/queries/0_stateless/02366_kql_func_binary.reference b/tests/queries/0_stateless/02366_kql_func_binary.reference index 6276cd6d867..360c1aa9899 100644 --- a/tests/queries/0_stateless/02366_kql_func_binary.reference +++ b/tests/queries/0_stateless/02366_kql_func_binary.reference @@ -1,7 +1,4 @@ -- binary functions 4 7 -1 -1 -1 7 3 1 diff --git a/tests/queries/0_stateless/02366_kql_func_binary.sql b/tests/queries/0_stateless/02366_kql_func_binary.sql index 824022b564c..687f3afb5ee 100644 --- a/tests/queries/0_stateless/02366_kql_func_binary.sql +++ b/tests/queries/0_stateless/02366_kql_func_binary.sql @@ -1,8 +1,5 @@ set dialect='kusto'; print ' -- binary functions'; print binary_and(4,7), binary_or(4,7); -print binary_shift_left(1, 1) == binary_shift_left(1, 65); -print binary_shift_right(2, 1) == binary_shift_right(2, 65); -print binary_shift_right(binary_shift_left(1, 65), 65) == 1; print binary_xor(2, 5), bitset_count_ones(42); print bitset_count_ones(binary_shift_left(binary_and(4,7), 1)); diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql index 6b2961f0555..91e8624057c 100644 --- a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS t1; CREATE TABLE t0 (vkey UInt32, pkey UInt32, c0 UInt32) engine = TinyLog; CREATE TABLE t1 (vkey UInt32) ENGINE = AggregatingMergeTree ORDER BY vkey; INSERT INTO t0 VALUES (15, 25000, 58); -SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); +SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); -- { serverError ARGUMENT_OUT_OF_BOUND } DROP TABLE t0; DROP TABLE t1; diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 31972894c3d..380656cd1ca 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -94,7 +94,6 @@ EXTERN_TYPES_EXCLUDES=( ErrorCodes::values[i] ErrorCodes::getErrorCodeByName ErrorCodes::Value - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT ) for extern_type in ${!EXTERN_TYPES[@]}; do type_of_extern=${EXTERN_TYPES[$extern_type]} From 12608d2090485e5cd98f82b78c64f7014e7e391c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 1 Jul 2024 14:22:32 +0000 Subject: [PATCH 043/107] Improve exception text --- src/Functions/bitShiftLeft.cpp | 6 +++--- src/Functions/bitShiftRight.cpp | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 3d496296ba9..645672c50e2 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -26,7 +26,7 @@ struct BitShiftLeftImpl if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); else if (b < 0 || static_cast(b) > 8 * sizeof(A)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) << static_cast(b); else @@ -43,7 +43,7 @@ struct BitShiftLeftImpl UInt8 word_size = 8; size_t n = end - pos; if (b < 0 || b > B(word_size * n)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); /// To prevent overflow if (static_cast(b) >= (static_cast(n) * word_size)) @@ -112,7 +112,7 @@ struct BitShiftLeftImpl UInt8 word_size = 8; size_t n = end - pos; if (b < 0 || b > B(word_size * n)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); /// To prevent overflow if (static_cast(b) >= (static_cast(n) * word_size)) { diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 0b41493fc6d..2e9182d3fe6 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -27,7 +27,7 @@ struct BitShiftRightImpl if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); else if (b < 0 || static_cast(b) > 8 * sizeof(A)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if constexpr (is_big_int_v) return static_cast(a) >> static_cast(b); else @@ -59,7 +59,7 @@ struct BitShiftRightImpl UInt8 word_size = 8; size_t n = end - pos; if (b < 0 || b > B(word_size * n)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); /// To prevent overflow if (static_cast(b) >= (static_cast(n) * word_size)) { @@ -99,7 +99,7 @@ struct BitShiftRightImpl UInt8 word_size = 8; size_t n = end - pos; if (b < 0 || b > B(word_size * n)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a positive value and not greater than the bit width of the value to shift"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); /// To prevent overflow if (static_cast(b) >= (static_cast(n) * word_size)) { From 7a7633a2309a5f1286f6120513dd75a54aefa1b6 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Jul 2024 15:17:56 +0000 Subject: [PATCH 044/107] Simplify logic in corner case to avoid comparing doubles The case for > bit_limit is already covered in previous branch, so we just need to cover the other case. This also fixes an overflow that was caused in previous check. e.g. b > B(word_size * n) if sizeof(B) is 1 byte but n is huge --- src/Functions/bitShiftLeft.cpp | 17 ++++++++--------- src/Functions/bitShiftRight.cpp | 16 ++++++++-------- ...ift_throws_error_for_out_of_bounds.reference | 2 ++ ...bit_shift_throws_error_for_out_of_bounds.sql | 2 ++ 4 files changed, 20 insertions(+), 17 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 645672c50e2..99fd11114aa 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -40,13 +40,12 @@ struct BitShiftLeftImpl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); else { - UInt8 word_size = 8; + const UInt8 word_size = 8 * sizeof(*pos); size_t n = end - pos; - if (b < 0 || b > B(word_size * n)) + const UInt256 bit_limit = word_size * n; + if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - - /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size)) + else if (b == bit_limit) { // insert default value out_vec.push_back(0); @@ -109,12 +108,12 @@ struct BitShiftLeftImpl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); else { - UInt8 word_size = 8; + const UInt8 word_size = 8; size_t n = end - pos; - if (b < 0 || b > B(word_size * n)) + const UInt256 bit_limit = word_size * n; + if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size)) + else if (b == bit_limit) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 2e9182d3fe6..bdc193c4be6 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -56,12 +56,12 @@ struct BitShiftRightImpl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); else { - UInt8 word_size = 8; + const UInt8 word_size = 8; size_t n = end - pos; - if (b < 0 || b > B(word_size * n)) + const UInt256 bit_limit = word_size * n; + if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size)) + else if (b == bit_limit) { /// insert default value out_vec.push_back(0); @@ -96,12 +96,12 @@ struct BitShiftRightImpl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); else { - UInt8 word_size = 8; + const UInt8 word_size = 8; size_t n = end - pos; - if (b < 0 || b > B(word_size * n)) + const UInt256 bit_limit = word_size * n; + if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - /// To prevent overflow - if (static_cast(b) >= (static_cast(n) * word_size)) + else if (b == bit_limit) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference index d86bac9de59..33b8cd6ee26 100644 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference @@ -1 +1,3 @@ +-- bitShiftRight +-- bitShiftLeft OK diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql index a1a246593d8..aec01753673 100644 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql @@ -1,3 +1,4 @@ +SELECT '-- bitShiftRight'; SELECT bitShiftRight(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftRight(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftRight('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } @@ -5,6 +6,7 @@ SELECT bitShiftRight('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND SELECT bitShiftRight(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT '-- bitShiftLeft'; SELECT bitShiftLeft(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftLeft(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT bitShiftLeft('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } From 21f0eb2eecab17f6137639dc9162dbc4301d0d95 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Jul 2024 18:13:49 +0200 Subject: [PATCH 045/107] Do not use async initialization of readers in s3queue --- .../StorageObjectStorageSource.cpp | 94 ++++++++----- .../StorageObjectStorageSource.h | 30 +++- .../ObjectStorageQueueIFileMetadata.cpp | 10 ++ .../ObjectStorageQueueIFileMetadata.h | 2 +- .../ObjectStorageQueueSource.cpp | 133 ++++++------------ .../ObjectStorageQueueSource.h | 36 ++--- .../StorageObjectStorageQueue.cpp | 41 +----- .../integration/test_storage_s3_queue/test.py | 1 + 8 files changed, 158 insertions(+), 189 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index aef783fc3c4..0f0aae7a6a5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -65,7 +65,6 @@ StorageObjectStorageSource::StorageObjectStorageSource( CurrentMetrics::StorageObjectStorageThreadsActive, CurrentMetrics::StorageObjectStorageThreadsScheduled, 1/* max_threads */)) - , columns_desc(info.columns_description) , file_iterator(file_iterator_) , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) @@ -156,20 +155,20 @@ std::shared_ptr StorageObjectStorageSourc return iterator; } -void StorageObjectStorageSource::lazyInitialize(size_t processor) +void StorageObjectStorageSource::lazyInitialize() { if (initialized) return; - reader = createReader(processor); + reader = createReader(); if (reader) - reader_future = createReaderAsync(processor); + reader_future = createReaderAsync(); initialized = true; } Chunk StorageObjectStorageSource::generate() { - lazyInitialize(0); + lazyInitialize(); while (true) { @@ -236,27 +235,30 @@ void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_inf schema_cache.addNumRows(cache_key, num_rows); } -std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfo & object_info) +StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader() { - const auto cache_key = getKeyForSchemaCache( - getUniqueStoragePathIdentifier(*configuration, object_info), - configuration->format, - format_settings, - getContext()); - - auto get_last_mod_time = [&]() -> std::optional - { - return object_info.metadata - ? std::optional(object_info.metadata->last_modified.epochTime()) - : std::nullopt; - }; - return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); + return createReader( + 0, file_iterator, configuration, object_storage, read_from_format_info, format_settings, + key_condition, getContext(), &schema_cache, log, max_block_size, max_parsing_threads, need_only_count); } -StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader(size_t processor) +StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader( + size_t processor, + const std::shared_ptr & file_iterator, + const ConfigurationPtr & configuration, + const ObjectStoragePtr & object_storage, + const ReadFromFormatInfo & read_from_format_info, + const std::optional & format_settings, + const std::shared_ptr & key_condition_, + const ContextPtr & context_, + SchemaCache * schema_cache, + const LoggerPtr & log, + size_t max_block_size, + size_t max_parsing_threads, + bool need_only_count) { ObjectInfoPtr object_info; - auto query_settings = configuration->getQuerySettings(getContext()); + auto query_settings = configuration->getQuerySettings(context_); do { @@ -277,9 +279,29 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade std::shared_ptr source; std::unique_ptr read_buf; + auto try_get_num_rows_from_cache = [&]() -> std::optional + { + if (!schema_cache) + return std::nullopt; + + const auto cache_key = getKeyForSchemaCache( + getUniqueStoragePathIdentifier(*configuration, *object_info), + configuration->format, + format_settings, + context_); + + auto get_last_mod_time = [&]() -> std::optional + { + return object_info->metadata + ? std::optional(object_info->metadata->last_modified.epochTime()) + : std::nullopt; + }; + return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); + }; + std::optional num_rows_from_cache = need_only_count - && getContext()->getSettingsRef().use_cache_for_count_from_files - ? tryGetNumRowsFromCache(*object_info) + && context_->getSettingsRef().use_cache_for_count_from_files + ? try_get_num_rows_from_cache() : std::nullopt; if (num_rows_from_cache) @@ -304,14 +326,14 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); - read_buf = createReadBuffer(*object_info); + read_buf = createReadBuffer(*object_info, object_storage, context_, log); } auto input_format = FormatFactory::instance().getInput( configuration->format, *read_buf, read_from_format_info.format_header, - getContext(), + context_, max_block_size, format_settings, need_only_count ? 1 : max_parsing_threads, @@ -320,20 +342,20 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade compression_method, need_only_count); - if (key_condition) - input_format->setKeyCondition(key_condition); + if (key_condition_) + input_format->setKeyCondition(key_condition_); if (need_only_count) input_format->needOnlyCount(); builder.init(Pipe(input_format)); - if (columns_desc.hasDefaults()) + if (read_from_format_info.columns_description.hasDefaults()) { builder.addSimpleTransform( [&](const Block & header) { - return std::make_shared(header, columns_desc, *input_format, getContext()); + return std::make_shared(header, read_from_format_info.columns_description, *input_format, context_); }); } @@ -356,21 +378,25 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade object_info, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)); } -std::future StorageObjectStorageSource::createReaderAsync(size_t processor) +std::future StorageObjectStorageSource::createReaderAsync() { - return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); + return create_reader_scheduler([=, this] { return createReader(); }, Priority{}); } -std::unique_ptr StorageObjectStorageSource::createReadBuffer(const ObjectInfo & object_info) +std::unique_ptr StorageObjectStorageSource::createReadBuffer( + const ObjectInfo & object_info, + const ObjectStoragePtr & object_storage, + const ContextPtr & context_, + const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; - const bool object_too_small = object_size <= 2 * getContext()->getSettings().max_download_buffer_size; + const bool object_too_small = object_size <= 2 * context_->getSettings().max_download_buffer_size; const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; /// User's object may change, don't cache it. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index d93097d2636..c2bfff4b997 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -75,7 +75,6 @@ protected: const ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; - ColumnsDescription columns_desc; std::shared_ptr file_iterator; SchemaCache & schema_cache; bool initialized = false; @@ -116,13 +115,32 @@ protected: std::future reader_future; /// Recreate ReadBuffer and Pipeline for each file. - ReaderHolder createReader(size_t processor = 0); - std::future createReaderAsync(size_t processor = 0); - std::unique_ptr createReadBuffer(const ObjectInfo & object_info); + static ReaderHolder createReader( + size_t processor, + const std::shared_ptr & file_iterator, + const ConfigurationPtr & configuration, + const ObjectStoragePtr & object_storage, + const ReadFromFormatInfo & read_from_format_info, + const std::optional & format_settings, + const std::shared_ptr & key_condition_, + const ContextPtr & context_, + SchemaCache * schema_cache, + const LoggerPtr & log, + size_t max_block_size, + size_t max_parsing_threads, + bool need_only_count); + + ReaderHolder createReader(); + + std::future createReaderAsync(); + static std::unique_ptr createReadBuffer( + const ObjectInfo & object_info, + const ObjectStoragePtr & object_storage, + const ContextPtr & context_, + const LoggerPtr & log); void addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows); - std::optional tryGetNumRowsFromCache(const ObjectInfo & object_info); - void lazyInitialize(size_t processor); + void lazyInitialize(); }; class StorageObjectStorageSource::IIterator diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.cpp index 52ee0c9f8ed..6fac519849d 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.cpp @@ -62,6 +62,11 @@ void ObjectStorageQueueIFileMetadata::FileStatus::onFailed(const std::string & e last_exception = exception; } +void ObjectStorageQueueIFileMetadata::FileStatus::updateState(State state_) +{ + state = state_; +} + std::string ObjectStorageQueueIFileMetadata::FileStatus::getException() const { std::lock_guard lock(last_exception_mutex); @@ -224,9 +229,14 @@ bool ObjectStorageQueueIFileMetadata::setProcessing() auto [success, file_state] = setProcessingImpl(); if (success) + { file_status->onProcessing(); + } else + { + LOG_TEST(log, "Updating state of {} from {} to {}", path, file_status->state.load(), file_state); file_status->updateState(file_state); + } LOG_TEST(log, "File {} has state `{}`: will {}process (processing id version: {})", path, file_state, success ? "" : "not ", diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h index 652b4742389..920beaa6f21 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h @@ -23,7 +23,7 @@ public: void onProcessing(); void onProcessed(); void onFailed(const std::string & exception); - void updateState(State state_) { state = state_; } + void updateState(State state_); std::string getException() const; diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 955e49bc2bf..683a7038bb6 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -357,41 +357,38 @@ ObjectStorageQueueSource::FileIterator::getNextKeyFromAcquiredBucket(size_t proc ObjectStorageQueueSource::ObjectStorageQueueSource( String name_, size_t processor_id_, - const Block & header_, - std::unique_ptr internal_source_, + std::shared_ptr file_iterator_, + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const ReadFromFormatInfo & read_from_format_info_, + const std::optional & format_settings_, + const ObjectStorageQueueSettings & queue_settings_, std::shared_ptr files_metadata_, - const ObjectStorageQueueAction & action_, - RemoveFileFunc remove_file_func_, - const NamesAndTypesList & requested_virtual_columns_, ContextPtr context_, + size_t max_block_size_, const std::atomic & shutdown_called_, const std::atomic & table_is_being_dropped_, std::shared_ptr system_queue_log_, const StorageID & storage_id_, LoggerPtr log_, - size_t max_processed_files_before_commit_, - size_t max_processed_rows_before_commit_, - size_t max_processed_bytes_before_commit_, - size_t max_processing_time_sec_before_commit_, bool commit_once_processed_) - : ISource(header_) + : ISource(read_from_format_info_.source_header) , WithContext(context_) , name(std::move(name_)) , processor_id(processor_id_) - , action(action_) + , file_iterator(file_iterator_) + , configuration(configuration_) + , object_storage(object_storage_) + , read_from_format_info(read_from_format_info_) + , format_settings(format_settings_) + , queue_settings(queue_settings_) , files_metadata(files_metadata_) - , internal_source(std::move(internal_source_)) - , requested_virtual_columns(requested_virtual_columns_) + , max_block_size(max_block_size_) , shutdown_called(shutdown_called_) , table_is_being_dropped(table_is_being_dropped_) , system_queue_log(system_queue_log_) , storage_id(storage_id_) - , max_processed_files_before_commit(max_processed_files_before_commit_) - , max_processed_rows_before_commit(max_processed_rows_before_commit_) - , max_processed_bytes_before_commit(max_processed_bytes_before_commit_) - , max_processing_time_sec_before_commit(max_processing_time_sec_before_commit_) , commit_once_processed(commit_once_processed_) - , remove_file_func(remove_file_func_) , log(log_) { } @@ -401,21 +398,6 @@ String ObjectStorageQueueSource::getName() const return name; } -void ObjectStorageQueueSource::lazyInitialize(size_t processor) -{ - if (initialized) - return; - - LOG_TEST(log, "Initializing a new reader"); - - internal_source->lazyInitialize(processor); - reader = std::move(internal_source->reader); - if (reader) - reader_future = std::move(internal_source->reader_future); - - initialized = true; -} - Chunk ObjectStorageQueueSource::generate() { Chunk chunk; @@ -440,14 +422,21 @@ Chunk ObjectStorageQueueSource::generate() Chunk ObjectStorageQueueSource::generateImpl() { - lazyInitialize(processor_id); - - while (true) + while (!shutdown_called) { if (!reader) { - LOG_TEST(log, "No reader"); - break; + const auto context = getContext(); + reader = StorageObjectStorageSource::createReader( + processor_id, file_iterator, configuration, object_storage, read_from_format_info, + format_settings, nullptr, context, nullptr, log, max_block_size, + context->getSettingsRef().max_parsing_threads.value, /* need_only_count */false); + + if (!reader) + { + LOG_TEST(log, "No reader"); + break; + } } const auto * object_info = dynamic_cast(reader.getObjectInfo().get()); @@ -528,7 +517,7 @@ Chunk ObjectStorageQueueSource::generateImpl() total_processed_bytes += chunk.bytes(); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( - chunk, requested_virtual_columns, + chunk, read_from_format_info.requested_virtual_columns, { .path = path, .size = reader.getObjectInfo()->metadata->size_bytes @@ -547,9 +536,6 @@ Chunk ObjectStorageQueueSource::generateImpl() if (processed_rows_from_file == 0) { - auto * file_iterator = dynamic_cast(internal_source->file_iterator.get()); - chassert(file_iterator); - if (file_status->retries < file_metadata->getMaxTries()) file_iterator->returnForRetry(reader.getObjectInfo()); @@ -564,11 +550,13 @@ Chunk ObjectStorageQueueSource::generateImpl() file_status->setProcessingEndTime(); file_status.reset(); + reader = {}; processed_rows_from_file = 0; processed_files.push_back(file_metadata); - if (processed_files.size() == max_processed_files_before_commit) + if (queue_settings.max_processed_files_before_commit + && processed_files.size() == queue_settings.max_processed_files_before_commit) { LOG_TRACE(log, "Number of max processed files before commit reached " "(rows: {}, bytes: {}, files: {})", @@ -576,68 +564,30 @@ Chunk ObjectStorageQueueSource::generateImpl() break; } - bool rows_or_bytes_or_time_limit_reached = false; - if (max_processed_rows_before_commit - && total_processed_rows == max_processed_rows_before_commit) + if (queue_settings.max_processed_rows_before_commit + && total_processed_rows == queue_settings.max_processed_rows_before_commit) { LOG_TRACE(log, "Number of max processed rows before commit reached " "(rows: {}, bytes: {}, files: {})", total_processed_rows, total_processed_bytes, processed_files.size()); - - rows_or_bytes_or_time_limit_reached = true; + break; } - else if (max_processed_bytes_before_commit - && total_processed_bytes == max_processed_bytes_before_commit) + else if (queue_settings.max_processed_bytes_before_commit + && total_processed_bytes == queue_settings.max_processed_bytes_before_commit) { LOG_TRACE(log, "Number of max processed bytes before commit reached " "(rows: {}, bytes: {}, files: {})", total_processed_rows, total_processed_bytes, processed_files.size()); - - rows_or_bytes_or_time_limit_reached = true; + break; } - else if (max_processing_time_sec_before_commit - && total_stopwatch.elapsedSeconds() >= max_processing_time_sec_before_commit) + else if (queue_settings.max_processing_time_sec_before_commit + && total_stopwatch.elapsedSeconds() >= queue_settings.max_processing_time_sec_before_commit) { LOG_TRACE(log, "Max processing time before commit reached " "(rows: {}, bytes: {}, files: {})", total_processed_rows, total_processed_bytes, processed_files.size()); - - rows_or_bytes_or_time_limit_reached = true; - } - - if (rows_or_bytes_or_time_limit_reached) - { - if (!reader_future.valid()) - break; - - LOG_TRACE(log, "Rows or bytes limit reached, but we have one more file scheduled already, " - "will process it despite the limit"); - } - - if (shutdown_called) - { - LOG_TRACE(log, "Shutdown was called, stopping sync"); break; } - - chassert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - { - LOG_TEST(log, "Reader finished"); - break; - } - - file_status = files_metadata->getFileStatus(reader.getObjectInfo()->getPath()); - - if (!rows_or_bytes_or_time_limit_reached && processed_files.size() + 1 < max_processed_files_before_commit) - { - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - internal_source->create_reader_pool->wait(); - reader_future = internal_source->createReaderAsync(processor_id); - } } return {}; @@ -681,12 +631,11 @@ void ObjectStorageQueueSource::commit(bool success, const std::string & exceptio void ObjectStorageQueueSource::applyActionAfterProcessing(const String & path) { - switch (action) + switch (queue_settings.after_processing.value) { case ObjectStorageQueueAction::DELETE: { - assert(remove_file_func); - remove_file_func(path); + object_storage->removeObject(StoredObject(path)); break; } case ObjectStorageQueueAction::KEEP: diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h index ccd87e8a269..fce2a426ecb 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h @@ -97,22 +97,20 @@ public: ObjectStorageQueueSource( String name_, size_t processor_id_, - const Block & header_, - std::unique_ptr internal_source_, + std::shared_ptr file_iterator_, + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const ReadFromFormatInfo & read_from_format_info_, + const std::optional & format_settings_, + const ObjectStorageQueueSettings & queue_settings_, std::shared_ptr files_metadata_, - const ObjectStorageQueueAction & action_, - RemoveFileFunc remove_file_func_, - const NamesAndTypesList & requested_virtual_columns_, ContextPtr context_, + size_t max_block_size_, const std::atomic & shutdown_called_, const std::atomic & table_is_being_dropped_, std::shared_ptr system_queue_log_, const StorageID & storage_id_, LoggerPtr log_, - size_t max_processed_files_before_commit_, - size_t max_processed_rows_before_commit_, - size_t max_processed_bytes_before_commit_, - size_t max_processing_time_sec_before_commit_, bool commit_once_processed_); static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); @@ -128,29 +126,27 @@ public: private: const String name; const size_t processor_id; - const ObjectStorageQueueAction action; + const std::shared_ptr file_iterator; + const ConfigurationPtr configuration; + const ObjectStoragePtr object_storage; + const ReadFromFormatInfo read_from_format_info; + const std::optional format_settings; + const ObjectStorageQueueSettings queue_settings; const std::shared_ptr files_metadata; - const std::shared_ptr internal_source; - const NamesAndTypesList requested_virtual_columns; + const size_t max_block_size; + const std::atomic & shutdown_called; const std::atomic & table_is_being_dropped; const std::shared_ptr system_queue_log; const StorageID storage_id; - const size_t max_processed_files_before_commit; - const size_t max_processed_rows_before_commit; - const size_t max_processed_bytes_before_commit; - const size_t max_processing_time_sec_before_commit; const bool commit_once_processed; - RemoveFileFunc remove_file_func; LoggerPtr log; std::vector processed_files; std::vector failed_during_read_files; Source::ReaderHolder reader; - std::future reader_future; - std::atomic initialized{false}; size_t processed_rows_from_file = 0; size_t total_processed_rows = 0; @@ -165,8 +161,6 @@ private: ObjectStorageQueueMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); - - void lazyInitialize(size_t processor); }; } diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 95265cde9ea..4388864434e 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -352,43 +352,14 @@ std::shared_ptr StorageObjectStorageQueue::createSourc ContextPtr local_context, bool commit_once_processed) { - auto internal_source = std::make_unique( - getName(), - object_storage, - configuration, - info, - format_settings, - local_context, - max_block_size, - file_iterator, - local_context->getSettingsRef().max_download_threads, - false); - - auto file_deleter = [=, this](const std::string & path) mutable - { - object_storage->removeObject(StoredObject(path)); - }; - return std::make_shared( - getName(), - processor_id, - info.source_header, - std::move(internal_source), - files_metadata, - queue_settings->after_processing, - file_deleter, - info.requested_virtual_columns, - local_context, - shutdown_called, - table_is_being_dropped, + getName(), processor_id, + file_iterator, configuration, object_storage, + info, format_settings, + *queue_settings, files_metadata, + local_context, max_block_size, shutdown_called, table_is_being_dropped, getQueueLog(object_storage, local_context, *queue_settings), - getStorageID(), - log, - queue_settings->max_processed_files_before_commit, - queue_settings->max_processed_rows_before_commit, - queue_settings->max_processed_bytes_before_commit, - queue_settings->max_processing_time_sec_before_commit, - commit_once_processed); + getStorageID(), log, commit_once_processed); } bool StorageObjectStorageQueue::hasDependencies(const StorageID & table_id) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b93e560d5b9..bf3c28c5429 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1780,6 +1780,7 @@ def test_commit_on_limit(started_cluster): if "test_999999.csv" in get_processed_files(): break time.sleep(1) + assert "test_999999.csv" in get_processed_files() assert 1 == int( From 389a86ec059461f20521e4dd2c7888e2b3b37623 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Jul 2024 16:20:42 +0000 Subject: [PATCH 046/107] Sort error codes alphabetically --- src/Functions/bitShiftLeft.cpp | 2 +- src/Functions/bitShiftRight.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 99fd11114aa..8e39ed86461 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -5,9 +5,9 @@ namespace DB { namespace ErrorCodes { + extern const int ARGUMENT_OUT_OF_BOUND; extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int ARGUMENT_OUT_OF_BOUND; } namespace diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index bdc193c4be6..46cfcde8a33 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -6,9 +6,9 @@ namespace DB { namespace ErrorCodes { + extern const int ARGUMENT_OUT_OF_BOUND; extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int ARGUMENT_OUT_OF_BOUND; } namespace From d2cade4aa38be9f33715d593cb2e0d549c9f565e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 2 Jul 2024 20:11:06 +0200 Subject: [PATCH 047/107] Relax the check in 02982_aggregation_states_destruction --- .../queries/0_stateless/02982_aggregation_states_destruction.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.sh b/tests/queries/0_stateless/02982_aggregation_states_destruction.sh index 263a4535c0e..84183606d48 100755 --- a/tests/queries/0_stateless/02982_aggregation_states_destruction.sh +++ b/tests/queries/0_stateless/02982_aggregation_states_destruction.sh @@ -11,4 +11,4 @@ $CLICKHOUSE_CLIENT --query_id $query_id --log_query_threads 1 --query="select nu $CLICKHOUSE_CLIENT -q "system flush logs;" -$CLICKHOUSE_CLIENT -q "select count() > 0, (countIf(thread_name = 'AggregDestruct') as aggs) > 0, aggs > 1 from system.query_thread_log where query_id = '$query_id' and current_database = currentDatabase();" +$CLICKHOUSE_CLIENT -q "select count() > 0 from system.query_thread_log where query_id = '$query_id' and current_database = currentDatabase() and thread_name = 'AggregDestruct';" From bcf8a93a52204cb80a867c237f302221bb51c272 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 3 Jul 2024 01:34:25 -0400 Subject: [PATCH 048/107] `max_query_length` argument for the fuzzQuery --- .../table-functions/fuzzQuery.md | 3 ++- src/Storages/StorageFuzzQuery.cpp | 20 +++++++++++++------ src/Storages/StorageFuzzQuery.h | 1 + .../03031_table_function_fuzzquery.sql | 4 ++-- 4 files changed, 19 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md index ff8cfd1cd3b..e15f8a40156 100644 --- a/docs/en/sql-reference/table-functions/fuzzQuery.md +++ b/docs/en/sql-reference/table-functions/fuzzQuery.md @@ -9,12 +9,13 @@ sidebar_label: fuzzQuery Perturbs the given query string with random variations. ``` sql -fuzzQuery(query[, random_seed]) +fuzzQuery(query[, max_query_length[, random_seed]]) ``` **Arguments** - `query` (String) - The source query to perform the fuzzing on. +- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process. - `random_seed` (UInt64) - A random seed for producing stable results. **Returned Value** diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp index 5e29a04427b..229ae1af7c1 100644 --- a/src/Storages/StorageFuzzQuery.cpp +++ b/src/Storages/StorageFuzzQuery.cpp @@ -47,7 +47,7 @@ ColumnPtr FuzzQuerySource::createColumn() size_t data_len = data.size(); /// AST is too long, will start from the original query. - if (data_len > 500) + if (config.max_query_length > 500) { fuzz_base = query; continue; @@ -120,10 +120,11 @@ StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine // Supported signatures: // - // FuzzQuery('query') - // FuzzQuery('query', 'random_seed') - if (engine_args.empty() || engine_args.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 2 arguments: query, random_seed"); + // FuzzQuery(query) + // FuzzQuery(query, max_query_length) + // FuzzQuery(query, max_query_length, random_seed) + if (engine_args.empty() || engine_args.size() > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 3 arguments: query, max_query_length, random_seed"); for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); @@ -131,9 +132,16 @@ StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); configuration.query = std::move(first_arg); - if (engine_args.size() == 2) + if (engine_args.size() >= 2) { const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.max_query_length = checkAndGetLiteralArgument(literal, "max_query_length"); + } + + if (engine_args.size() == 3) + { + const auto & literal = engine_args[2]->as(); if (!literal.value.isNull()) configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); } diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h index 3ae506fdfb8..125ef960e74 100644 --- a/src/Storages/StorageFuzzQuery.h +++ b/src/Storages/StorageFuzzQuery.h @@ -18,6 +18,7 @@ public: struct Configuration : public StatelessTableEngineConfiguration { String query; + UInt64 max_query_length = 500; UInt64 random_seed = randomSeed(); }; diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql index 5821e2e5111..b26096f7f0e 100644 --- a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -1,5 +1,5 @@ -SELECT * FROM fuzzQuery('SELECT 1', 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; +SELECT * FROM fuzzQuery('SELECT 1', 500, 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; SELECT * FROM fuzzQuery('SELECT * FROM ( @@ -15,4 +15,4 @@ FROM ( ) AS r ON l.item_id = r.item_id ORDER BY 1,2,3; -', 8956) LIMIT 10 FORMAT NULL; +', 500, 8956) LIMIT 10 FORMAT NULL; From cfafbc388cb1ac3ca7c5c6810f3f5c00f3b8b3d5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 3 Jul 2024 11:56:07 +0200 Subject: [PATCH 049/107] Fix test_drop_table --- .../ObjectStorageQueue/ObjectStorageQueueSource.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 683a7038bb6..dc5fb6d2744 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -422,8 +422,14 @@ Chunk ObjectStorageQueueSource::generate() Chunk ObjectStorageQueueSource::generateImpl() { - while (!shutdown_called) + while (true) { + if (shutdown_called) + { + LOG_TRACE(log, "Shutdown was called, stopping sync"); + break; + } + if (!reader) { const auto context = getContext(); From 5fd36059e48c4377ea526343e7272009d2ccaf2a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Jul 2024 15:28:01 +0200 Subject: [PATCH 050/107] Try disabling background threads --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 023fdcf103a..cc5a391676f 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -34,7 +34,7 @@ if (OS_LINUX) # avoid spurious latencies and additional work associated with # MADV_DONTNEED. See # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000,prof:true,prof_active:false,background_thread:true") + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000,prof:true,prof_active:false") else() set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000") endif() From c96e3c6d1a9edb1e0d22cf818994f7647fa4a9d2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Jul 2024 16:21:36 +0200 Subject: [PATCH 051/107] Fix which I don't understand --- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 11 +++++++++-- ...3198_non_adaptive_granularity_no_errors.reference | 2 ++ .../03198_non_adaptive_granularity_no_errors.sql | 12 ++++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference create mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 9666f310d3d..a69d21de8e7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -560,7 +560,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai { /// With fixed granularity we can have last mark with less rows than granularity const bool is_last_mark = (mark_num + 1 == index_granularity.getMarksCount()); - if (!data_part->index_granularity_info.fixed_index_granularity || !is_last_mark) + if (!index_granularity_info.fixed_index_granularity || !is_last_mark) throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" @@ -785,7 +785,7 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_ /// We can adjust marks only if we computed granularity for blocks. /// Otherwise we cannot change granularity because it will differ from /// other columns -// if (compute_granularity && settings.can_use_adaptive_granularity) + if (compute_granularity && settings.can_use_adaptive_granularity) { if (getCurrentMark() != index_granularity.getMarksCount() - 1) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -824,7 +824,14 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_ /// Without offset rows_written_in_last_mark = 0; } + + if (compute_granularity) + { + index_granularity.popMark(); + index_granularity.appendMark(new_rows_in_last_mark); + } } + } } diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference new file mode 100644 index 00000000000..fcd78da1283 --- /dev/null +++ b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference @@ -0,0 +1,2 @@ +1000000 +1000000 diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql new file mode 100644 index 00000000000..25798ef6d33 --- /dev/null +++ b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS data_02051__fuzz_24; + +CREATE TABLE data_02051__fuzz_24 (`key` Int16, `value` String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part=0 AS SELECT number, repeat(toString(number), 5) FROM numbers(1000000.); + +SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); + +detach table data_02051__fuzz_24; +attach table data_02051__fuzz_24; + +SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); + +DROP TABLE data_02051__fuzz_24; From 9a023744a5825d349c1027e1c1d425956a3bc87f Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 3 Jul 2024 11:13:39 -0400 Subject: [PATCH 052/107] fix build --- src/Storages/StorageFuzzQuery.cpp | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp index 229ae1af7c1..6e8f425f8dc 100644 --- a/src/Storages/StorageFuzzQuery.cpp +++ b/src/Storages/StorageFuzzQuery.cpp @@ -1,14 +1,12 @@ #include #include -#include #include #include #include #include #include #include -#include #include #include @@ -41,10 +39,8 @@ ColumnPtr FuzzQuerySource::createColumn() fuzzer.fuzzMain(new_query); auto fuzzed_text = new_query->formatForErrorMessage(); - WriteBufferFromOwnString out; - formatAST(*new_query, out, false); - auto data = out.str(); - size_t data_len = data.size(); + if (base_before_fuzz == fuzzed_text) + continue; /// AST is too long, will start from the original query. if (config.max_query_length > 500) @@ -53,12 +49,12 @@ ColumnPtr FuzzQuerySource::createColumn() continue; } - IColumn::Offset next_offset = offset + data_len + 1; + IColumn::Offset next_offset = offset + fuzzed_text.size() + 1; data_to.resize(next_offset); - std::copy(data.begin(), data.end(), &data_to[offset]); + std::copy(fuzzed_text.begin(), fuzzed_text.end(), &data_to[offset]); - data_to[offset + data_len] = 0; + data_to[offset + fuzzed_text.size()] = 0; offsets_to[row_num] = next_offset; offset = next_offset; From 6aa2f7d5a0997fd56432f4550b3353e5d7cc6898 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 17:02:00 +0100 Subject: [PATCH 053/107] adjust reference file --- .../0_stateless/02982_aggregation_states_destruction.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.reference b/tests/queries/0_stateless/02982_aggregation_states_destruction.reference index 72749c905a3..d00491fd7e5 100644 --- a/tests/queries/0_stateless/02982_aggregation_states_destruction.reference +++ b/tests/queries/0_stateless/02982_aggregation_states_destruction.reference @@ -1 +1 @@ -1 1 1 +1 From 045cb0a5819e6e41198cd9cb05e27fb04aa08269 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Jul 2024 12:52:25 +0200 Subject: [PATCH 054/107] Increase special allocation sampling --- src/Common/GWPAsan.cpp | 2 +- src/Core/ServerSettings.h | 2 +- src/IO/BufferWithOwnMemory.h | 15 +++++++++++++-- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 0482ddb4e2b..48fbd07ec34 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 8000; + opts.SampleRate = 10000; const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 68ac45fa24f..e70be61118a 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -153,7 +153,7 @@ namespace DB M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ - M(Double, gwp_asan_force_sample_probability, 0, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ + M(Double, gwp_asan_force_sample_probability, 0.001, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 5c9a69893df..0ec733f7840 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -4,12 +4,15 @@ #include #include +#include #include #include #include +#include "config.h" + namespace ProfileEvents { @@ -41,10 +44,13 @@ struct Memory : boost::noncopyable, Allocator char * m_data = nullptr; size_t alignment = 0; + [[maybe_unused]] bool allow_gwp_asan_force_sample; + Memory() = default; /// If alignment != 0, then allocate memory aligned to specified value. - explicit Memory(size_t size_, size_t alignment_ = 0) : alignment(alignment_) + explicit Memory(size_t size_, size_t alignment_ = 0, bool allow_gwp_asan_force_sample_ = false) + : alignment(alignment_), allow_gwp_asan_force_sample(allow_gwp_asan_force_sample_) { alloc(size_); } @@ -127,6 +133,11 @@ private: ProfileEvents::increment(ProfileEvents::IOBufferAllocs); ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, new_capacity); +#if USE_GWP_ASAN + if (unlikely(allow_gwp_asan_force_sample && GWPAsan::shouldForceSample())) + gwp_asan::getThreadLocals()->NextSampleCounter = 1; +#endif + m_data = static_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; m_size = new_size; @@ -154,7 +165,7 @@ protected: public: /// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership. explicit BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment) + : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment, /*allow_gwp_asan_force_sample_=*/true) { Base::set(existing_memory ? existing_memory : memory.data(), size); Base::padded = !existing_memory; From 877445c88d71caffc79e6e7cd956298e84e9867e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 19:21:12 +0000 Subject: [PATCH 055/107] Fix reading dynamic subcolumns from altered Memory table --- src/Interpreters/getColumnFromBlock.cpp | 30 +++++++++++++++++++ src/Interpreters/getColumnFromBlock.h | 1 + .../QueryPlan/ReadFromMemoryStorageStep.cpp | 20 +++++++++---- ...3200_memory_engine_alter_dynamic.reference | 10 +++++++ .../03200_memory_engine_alter_dynamic.sql | 7 +++++ 5 files changed, 62 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference create mode 100644 tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql diff --git a/src/Interpreters/getColumnFromBlock.cpp b/src/Interpreters/getColumnFromBlock.cpp index 972e109afb3..2e70a58b5a1 100644 --- a/src/Interpreters/getColumnFromBlock.cpp +++ b/src/Interpreters/getColumnFromBlock.cpp @@ -31,6 +31,36 @@ ColumnPtr tryGetColumnFromBlock(const Block & block, const NameAndTypePair & req return castColumn({elem_column, elem_type, ""}, requested_column.type); } +ColumnPtr tryGetSubcolumnFromBlock(const Block & block, const DataTypePtr & requested_column_type, const NameAndTypePair & requested_subcolumn) +{ + const auto * elem = block.findByName(requested_subcolumn.getNameInStorage()); + if (!elem) + return nullptr; + + auto subcolumn_name = requested_subcolumn.getSubcolumnName(); + /// If requested subcolumn is dynamic, we should first perform cast and then + /// extract the subcolumn, because the data of dynamic subcolumn can change after cast. + if (elem->type->hasDynamicSubcolumns() && !elem->type->equals(*requested_column_type)) + { + auto casted_column = castColumn({elem->column, elem->type, ""}, requested_column_type); + auto elem_column = requested_column_type->tryGetSubcolumn(subcolumn_name, casted_column); + auto elem_type = requested_column_type->tryGetSubcolumnType(subcolumn_name); + + if (!elem_type || !elem_column) + return nullptr; + + return elem_column; + } + + auto elem_column = elem->type->tryGetSubcolumn(subcolumn_name, elem->column); + auto elem_type = elem->type->tryGetSubcolumnType(subcolumn_name); + + if (!elem_type || !elem_column) + return nullptr; + + return castColumn({elem_column, elem_type, ""}, requested_subcolumn.type); +} + ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & requested_column) { auto result_column = tryGetColumnFromBlock(block, requested_column); diff --git a/src/Interpreters/getColumnFromBlock.h b/src/Interpreters/getColumnFromBlock.h index 26500cfdd17..737ce9db555 100644 --- a/src/Interpreters/getColumnFromBlock.h +++ b/src/Interpreters/getColumnFromBlock.h @@ -9,5 +9,6 @@ namespace DB ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & requested_column); ColumnPtr tryGetColumnFromBlock(const Block & block, const NameAndTypePair & requested_column); +ColumnPtr tryGetSubcolumnFromBlock(const Block & block, const DataTypePtr & requested_column_type, const NameAndTypePair & requested_subcolumn); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 2e7693b1b36..6dc0c021a14 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -30,12 +30,15 @@ public: std::shared_ptr> parallel_execution_index_, InitializerFunc initializer_func_ = {}) : ISource(storage_snapshot->getSampleBlockForColumns(column_names_)) - , column_names_and_types(storage_snapshot->getColumnsByNames( + , requested_column_names_and_types(storage_snapshot->getColumnsByNames( GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withExtendedObjects(), column_names_)) , data(data_) , parallel_execution_index(parallel_execution_index_) , initializer_func(std::move(initializer_func_)) { + auto all_column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withExtendedObjects()); + for (const auto & [name, type] : all_column_names_and_types) + all_names_to_types[name] = type; } String getName() const override { return "Memory"; } @@ -59,17 +62,20 @@ protected: const Block & src = (*data)[current_index]; Columns columns; - size_t num_columns = column_names_and_types.size(); + size_t num_columns = requested_column_names_and_types.size(); columns.reserve(num_columns); - auto name_and_type = column_names_and_types.begin(); + auto name_and_type = requested_column_names_and_types.begin(); for (size_t i = 0; i < num_columns; ++i) { - columns.emplace_back(tryGetColumnFromBlock(src, *name_and_type)); + if (name_and_type->isSubcolumn()) + columns.emplace_back(tryGetSubcolumnFromBlock(src, all_names_to_types[name_and_type->getNameInStorage()], *name_and_type)); + else + columns.emplace_back(tryGetColumnFromBlock(src, *name_and_type)); ++name_and_type; } - fillMissingColumns(columns, src.rows(), column_names_and_types, column_names_and_types, {}, nullptr); + fillMissingColumns(columns, src.rows(), requested_column_names_and_types, requested_column_names_and_types, {}, nullptr); assert(std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column != nullptr; })); return Chunk(std::move(columns), src.rows()); @@ -88,7 +94,9 @@ private: } } - const NamesAndTypesList column_names_and_types; + const NamesAndTypesList requested_column_names_and_types; + /// Map (name -> type) for all columns from the storage header. + std::unordered_map all_names_to_types; size_t execution_index = 0; std::shared_ptr data; std::shared_ptr> parallel_execution_index; diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference new file mode 100644 index 00000000000..6d2c1334d6e --- /dev/null +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference @@ -0,0 +1,10 @@ +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql new file mode 100644 index 00000000000..95823283812 --- /dev/null +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql @@ -0,0 +1,7 @@ +set allow_experimental_dynamic_type=1; +create table test (d Dynamic) engine=Memory; +insert into table test select * from numbers(5); +alter table test modify column d Dynamic(max_types=1); +select d.UInt64 from test settings allow_experimental_analyzer=1; +select d.UInt64 from test settings allow_experimental_analyzer=1; + From 1b6ef06a91cf31b2aa8dbe5e3494ec63d602e4c9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 21:14:28 +0100 Subject: [PATCH 056/107] review fixes --- src/Common/CgroupsMemoryUsageObserver.cpp | 26 ++++++++++++----------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 33393a8b9c6..d36c7fd08aa 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -1,5 +1,3 @@ -#include -#include #include #if defined(OS_LINUX) @@ -14,7 +12,9 @@ #include #include +#include #include +#include #include #include "config.h" @@ -59,9 +59,9 @@ uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & ke } assertChar(' ', buf); - uint64_t mem_usage = 0; - readIntText(mem_usage, buf); - return mem_usage; + uint64_t value = 0; + readIntText(value, buf); + return value; } throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); @@ -96,10 +96,12 @@ struct CgroupsV2Reader : ICgroupsReader current_buf.rewind(); stat_buf.rewind(); - uint64_t mem_usage = 0; + int64_t mem_usage = 0; /// memory.current contains a single number + /// the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 readIntText(mem_usage, current_buf); mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file"); + chassert(mem_usage >= 0, "Negative memory usage"); return mem_usage; } @@ -153,13 +155,13 @@ std::optional getCgroupsV1Path() std::pair getCgroupsPath() { - auto v2_file_name = getCgroupsV2Path(); - if (v2_file_name.has_value()) - return {*v2_file_name, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; + auto v2_path = getCgroupsV2Path(); + if (v2_path.has_value()) + return {*v2_path, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; - auto v1_file_name = getCgroupsV1Path(); - if (v1_file_name.has_value()) - return {*v1_file_name, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; + auto v1_path = getCgroupsV1Path(); + if (v1_path.has_value()) + return {*v1_path, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); } From 5fb0fa3c3d3e611944f83ac06aaac2d6e5c0d0db Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Jul 2024 20:21:37 +0000 Subject: [PATCH 057/107] Automatic style fix --- tests/integration/test_memory_limit_observer/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_memory_limit_observer/test.py b/tests/integration/test_memory_limit_observer/test.py index 2840c830396..0eda165b1d2 100644 --- a/tests/integration/test_memory_limit_observer/test.py +++ b/tests/integration/test_memory_limit_observer/test.py @@ -76,4 +76,4 @@ def test_memory_usage_doesnt_include_page_cache_size(started_cluster): WHERE logger_name = 'CgroupsMemoryUsageObserver' AND message LIKE 'Read current memory usage%bytes%' """ ).strip() - assert int(max_mem_usage_from_cgroup) < 2 * 2 ** 30 + assert int(max_mem_usage_from_cgroup) < 2 * 2**30 From fe42d2ffe49addd28716585a4e2f3b8a53a3d6b0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Jul 2024 08:10:40 +0100 Subject: [PATCH 058/107] Lower sampling rate --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index e70be61118a..6c62ab6def8 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -153,7 +153,7 @@ namespace DB M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ - M(Double, gwp_asan_force_sample_probability, 0.001, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ + M(Double, gwp_asan_force_sample_probability, 0.0005, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp From e38e105e20ae6406a60baa0a08beed518676b346 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 16:44:20 +0800 Subject: [PATCH 059/107] add window function percent_rank --- .../sql-reference/window-functions/index.md | 1 + src/Processors/Transforms/WindowTransform.cpp | 171 ++++++++++++++++-- .../01592_window_functions.reference | 12 ++ .../0_stateless/01592_window_functions.sql | 18 ++ 4 files changed, 184 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 49076f3cbe1..8097abc0b15 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,6 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | +| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficent| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index b9f61d30182..0c7caca9de5 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,6 +16,10 @@ #include #include #include +#include "WindowTransform.h" + +#include +#include #include @@ -1609,8 +1613,37 @@ struct WindowFunctionHelpers { recurrent_detail::setValueToOutputColumn(transform, function_index, value); } + + ALWAYS_INLINE static bool checkPartitionEnterFirstRow(const WindowTransform * transform) { return transform->current_row_number == 1; } + + ALWAYS_INLINE static bool checkPartitionEnterLastRow(const WindowTransform * transform) + { + /// when partition_ended is false, it means that we don't reach the last row in this partition. + /// But when partition_ended is true, it doesn't mean that we reach the last row in this partition. + /// partition_ended is true when + /// - the input has finished. or + /// - current block contains next partition's data. + /// This is for fast check. + if (!transform->partition_ended) + return false; + + auto current_row = transform->current_row; + current_row.row++; + const auto & partitoin_end_row = transform->partition_end; + /// If current_row == partitoin_end_row, return true. otherwise + if (current_row != partitoin_end_row) + { + if (current_row.row < transform->blockRowsNumber(current_row)) + return false; + /// Next row to current_row may belong to next block. + if (partitoin_end_row.block != current_row.block + 1 || partitoin_end_row.row) + return false; + } + return true; + } }; + template struct StatefulWindowFunction : public WindowFunction { @@ -1639,6 +1672,8 @@ struct StatefulWindowFunction : public WindowFunction { return *reinterpret_cast(workspace.aggregate_function_state.data()); } + + }; struct ExponentialTimeDecayedSumState @@ -2128,7 +2163,7 @@ namespace } } // new partition - if (transform->current_row_number == 1) [[unlikely]] + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] { current_partition_rows = 0; current_partition_inserted_row = 0; @@ -2137,25 +2172,9 @@ namespace current_partition_rows++; // Only do the action when we meet the last row in this partition. - if (!transform->partition_ended) + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) return; - else - { - auto current_row = transform->current_row; - current_row.row++; - const auto & end_row = transform->partition_end; - if (current_row != end_row) - { - if (current_row.row < transform->blockRowsNumber(current_row)) - return; - if (end_row.block != current_row.block + 1 || end_row.row) - { - return; - } - // else, current_row is the last input row. - } - } auto bucket_capacity = current_partition_rows / buckets; auto capacity_diff = current_partition_rows - bucket_capacity * buckets; @@ -2211,6 +2230,115 @@ namespace } } +namespace +{ +struct PercentRankState +{ + RowNumber start_row; + UInt64 current_partition_rows = 0; +}; +} + +struct WindowFunctionPercentRank final : public StatefulWindowFunction +{ +public: + WindowFunctionPercentRank(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) + : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) + {} + + bool allocatesMemoryInArena() const override { return false; } + + std::optional getDefaultFrame() const override + { + WindowFrame frame; + frame.type = WindowFrame::FrameType::ROWS; + frame.end_type = WindowFrame::BoundaryType::Unbounded; + return frame; + } + + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + { + checkFrameBoundType(transform); + + auto & state = getWorkspaceState(transform, function_index); + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) + { + state.current_partition_rows = 0; + state.start_row = transform->current_row; + } + + insertRankIntoColumn(transform, function_index); + state.current_partition_rows++; + + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) + { + return; + } + + UInt64 remaining_rows = state.current_partition_rows; + Float64 percent_rank_denominator = state.current_partition_rows - 1; + + if (remaining_rows <= 1) + return; + while(remaining_rows > 0) + { + auto block_rows_number = transform->blockRowsNumber(state.start_row); + auto available_block_rows = block_rows_number - state.start_row.row; + if (available_block_rows <= remaining_rows) + { + auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; + auto & data = assert_cast(to_column).getData(); + for (size_t i = state.start_row.row; i < block_rows_number; ++i) + data[i] = data[i] / percent_rank_denominator; + + state.start_row.block++; + state.start_row.row = 0; + remaining_rows -= available_block_rows; + } + else + { + auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; + auto & data = assert_cast(to_column).getData(); + for (size_t i = state.start_row.row, n = state.start_row.row + remaining_rows; i < n; ++i) + { + data[i] = data[i]/percent_rank_denominator; + } + state.start_row.row += remaining_rows; + remaining_rows = 0; + } + } + } + + + inline PercentRankState & getWorkspaceState(const WindowTransform * transform, size_t function_index) const + { + const auto & workspace = transform->workspaces[function_index]; + return getState(workspace); + } + + inline void insertRankIntoColumn(const WindowTransform * transform, size_t function_index) const + { + auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; + assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); + } +private: + mutable bool has_check_frame_bound_type = false; + ALWAYS_INLINE void checkFrameBoundType(const WindowTransform * transform) const + { + if (has_check_frame_bound_type) + return; + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + } + has_check_frame_bound_type = true; + } +}; + // ClickHouse-specific variant of lag/lead that respects the window frame. template struct WindowFunctionLagLeadInFrame final : public WindowFunction @@ -2582,6 +2710,13 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) parameters); }, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("percent_rank", {[](const std::string & name, + const DataTypes & argument_types, const Array & parameters, const Settings *) + { + return std::make_shared(name, argument_types, + parameters); + }, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index ec957dd7a02..0995def71e2 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -79,3 +79,15 @@ iPhone 900 Smartphone 500 500 Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 +---- Q8 ---- +Lenovo Thinkpad Laptop 700 0 +Sony VAIO Laptop 700 0.3333333333333333 +Dell Vostro Laptop 800 0.6666666666666666 +HP Elite Laptop 1200 1 +Microsoft Lumia Smartphone 200 0 +HTC One Smartphone 400 0.3333333333333333 +Nexus Smartphone 500 0.6666666666666666 +iPhone Smartphone 900 1 +Kindle Fire Tablet 150 0 +Samsung Galaxy Tab Tablet 200 0.5 +iPad Tablet 700 1 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index f0d173b1f20..e48e26b26d2 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -101,5 +101,23 @@ SELECT FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; +select '---- Q8 ----'; +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS r + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + r ASC, + product_name ASC, + price ASC; + drop table product_groups; drop table products; From 6e231eedcf04e4136ea56fcbbb4a43916241dd23 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 17:16:20 +0800 Subject: [PATCH 060/107] fixed style --- src/Processors/Transforms/WindowTransform.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 0c7caca9de5..4758d5ca7f4 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,10 +16,6 @@ #include #include #include -#include "WindowTransform.h" - -#include -#include #include @@ -1643,7 +1639,6 @@ struct WindowFunctionHelpers } }; - template struct StatefulWindowFunction : public WindowFunction { @@ -1672,8 +1667,6 @@ struct StatefulWindowFunction : public WindowFunction { return *reinterpret_cast(workspace.aggregate_function_state.data()); } - - }; struct ExponentialTimeDecayedSumState @@ -2281,7 +2274,7 @@ public: if (remaining_rows <= 1) return; - while(remaining_rows > 0) + while (remaining_rows > 0) { auto block_rows_number = transform->blockRowsNumber(state.start_row); auto available_block_rows = block_rows_number - state.start_row.row; From 37d2ced74cd173c44015a89fcb9522ef9c3979ee Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 17:35:43 +0800 Subject: [PATCH 061/107] fixed typos --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 8097abc0b15..814a7ac4aca 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,7 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficent| +| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficient| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From 13d5b336adb8021742f842bff457ee8b3267b743 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 09:16:31 +0800 Subject: [PATCH 062/107] check window frame --- src/Processors/Transforms/WindowTransform.cpp | 91 +++++++++++-------- 1 file changed, 55 insertions(+), 36 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 4758d5ca7f4..45c9f4457b8 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -17,6 +17,9 @@ #include #include +#include +#include + #include @@ -71,6 +74,9 @@ public: size_t function_index) const = 0; virtual std::optional getDefaultFrame() const { return {}; } + + /// Is the frame type supported by this function. + virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } }; // Compares ORDER BY column values at given rows to find the boundaries of frame: @@ -402,6 +408,19 @@ WindowTransform::WindowTransform(const Block & input_header_, } } } + + for (const auto & workspace : workspaces) + { + if (workspace.window_function_impl) + { + if (!workspace.window_function_impl->checkWindowFrameType(this)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", + workspace.aggregate_function->getName()); + } + } + + } } WindowTransform::~WindowTransform() @@ -2086,8 +2105,6 @@ namespace const WindowTransform * transform, size_t function_index, const DataTypes & argument_types); - - static void checkWindowFrameType(const WindowTransform * transform); }; } @@ -2107,6 +2124,29 @@ struct WindowFunctionNtile final : public StatefulWindowFunction } bool allocatesMemoryInArena() const override { return false; } + + bool checkWindowFrameType(const WindowTransform * transform) const override + { + if (transform->order_by_indices.empty()) + { + LOG_ERROR(getLogger("WindowFunctionNtile"), "Window frame for 'ntile' function must have ORDER BY clause"); + return false; + } + + // We must wait all for the partition end and get the total rows number in this + // partition. So before the end of this partition, there is no any block could be + // dropped out. + bool is_frame_supported = transform->window_description.frame.begin_type == WindowFrame::BoundaryType::Unbounded + && transform->window_description.frame.end_type == WindowFrame::BoundaryType::Unbounded; + if (!is_frame_supported) + { + LOG_ERROR( + getLogger("WindowFunctionNtile"), + "Window frame for function 'ntile' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + return false; + } + return true; + } std::optional getDefaultFrame() const override { @@ -2134,7 +2174,6 @@ namespace { if (!buckets) [[unlikely]] { - checkWindowFrameType(transform); const auto & current_block = transform->blockAt(transform->current_row); const auto & workspace = transform->workspaces[function_index]; const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; @@ -2205,22 +2244,6 @@ namespace bucket_num += 1; } } - - void NtileState::checkWindowFrameType(const WindowTransform * transform) - { - if (transform->order_by_indices.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for 'ntile' function must have ORDER BY clause"); - - // We must wait all for the partition end and get the total rows number in this - // partition. So before the end of this partition, there is no any block could be - // dropped out. - bool is_frame_supported = transform->window_description.frame.begin_type == WindowFrame::BoundaryType::Unbounded - && transform->window_description.frame.end_type == WindowFrame::BoundaryType::Unbounded; - if (!is_frame_supported) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for function 'ntile' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - } - } } namespace @@ -2249,11 +2272,22 @@ public: frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } + + bool checkWindowFrameType(const WindowTransform * transform) const override + { + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + { + LOG_ERROR(getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + return false; + } + return true; + } + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { - checkFrameBoundType(transform); - auto & state = getWorkspaceState(transform, function_index); if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) { @@ -2315,21 +2349,6 @@ public: auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); } -private: - mutable bool has_check_frame_bound_type = false; - ALWAYS_INLINE void checkFrameBoundType(const WindowTransform * transform) const - { - if (has_check_frame_bound_type) - return; - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - } - has_check_frame_bound_type = true; - } }; // ClickHouse-specific variant of lag/lead that respects the window frame. From 04e7b11a6477ed8b554a12ca301d00ba01e0525d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 10:33:44 +0800 Subject: [PATCH 063/107] fixed style --- src/Processors/Transforms/WindowTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 45c9f4457b8..1cb447bb6d3 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2124,7 +2124,7 @@ struct WindowFunctionNtile final : public StatefulWindowFunction } bool allocatesMemoryInArena() const override { return false; } - + bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->order_by_indices.empty()) @@ -2272,7 +2272,7 @@ public: frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } - + bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded From 7f706dd9d1e7bf0b982c9db86f73c1cc89a4a0a5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:40:58 +0800 Subject: [PATCH 064/107] fixed --- src/Processors/Transforms/WindowTransform.cpp | 11 +--------- .../01592_window_functions.reference | 22 +++++++++---------- .../0_stateless/01592_window_functions.sql | 8 +++---- 3 files changed, 16 insertions(+), 25 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1cb447bb6d3..ce188ed47ae 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2265,18 +2265,9 @@ public: bool allocatesMemoryInArena() const override { return false; } - std::optional getDefaultFrame() const override - { - WindowFrame frame; - frame.type = WindowFrame::FrameType::ROWS; - frame.end_type = WindowFrame::BoundaryType::Unbounded; - return frame; - } - bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) { LOG_ERROR(getLogger("WindowFunctionPercentRank"), "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index 0995def71e2..f88360abcc1 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -80,14 +80,14 @@ Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 ---- Q8 ---- -Lenovo Thinkpad Laptop 700 0 -Sony VAIO Laptop 700 0.3333333333333333 -Dell Vostro Laptop 800 0.6666666666666666 -HP Elite Laptop 1200 1 -Microsoft Lumia Smartphone 200 0 -HTC One Smartphone 400 0.3333333333333333 -Nexus Smartphone 500 0.6666666666666666 -iPhone Smartphone 900 1 -Kindle Fire Tablet 150 0 -Samsung Galaxy Tab Tablet 200 0.5 -iPad Tablet 700 1 +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index e48e26b26d2..f4b868c36e4 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -109,15 +109,15 @@ FROM product_name, group_name, price, - percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS r + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent FROM products INNER JOIN product_groups USING (group_id) ) AS t ORDER BY group_name ASC, - r ASC, - product_name ASC, - price ASC; + price ASC, + product_name ASC; drop table product_groups; drop table products; From 656a9a7260e3789b8fb671b788dbb3126d88ebe9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:42:24 +0800 Subject: [PATCH 065/107] update --- tests/queries/0_stateless/01592_window_functions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index f4b868c36e4..2fc0e55bf02 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -109,14 +109,14 @@ FROM product_name, group_name, price, - rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent FROM products INNER JOIN product_groups USING (group_id) ) AS t ORDER BY group_name ASC, - price ASC, + price ASC, product_name ASC; drop table product_groups; From 91d2e5c72b38bb607d5a75020d38230a6937f310 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:48:42 +0800 Subject: [PATCH 066/107] more corver case --- tests/queries/0_stateless/01592_window_functions.reference | 1 + tests/queries/0_stateless/01592_window_functions.sql | 3 +++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index f88360abcc1..06ec67ee82d 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -91,3 +91,4 @@ iPhone Smartphone 900 4 1 Kindle Fire Tablet 150 1 0 Samsung Galaxy Tab Tablet 200 2 0.5 iPad Tablet 700 3 1 +Others Unknow 200 1 0 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index 2fc0e55bf02..a660fcca7b2 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -102,6 +102,9 @@ FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; select '---- Q8 ----'; +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + SELECT * FROM ( From e52828abf91c2e407fbdf5371e5c794a31b86b1e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 13 May 2024 11:19:33 +0800 Subject: [PATCH 067/107] fixed typos --- src/Processors/Transforms/WindowTransform.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index ce188ed47ae..ad592613da2 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1644,14 +1644,14 @@ struct WindowFunctionHelpers auto current_row = transform->current_row; current_row.row++; - const auto & partitoin_end_row = transform->partition_end; + const auto & partition_end_row = transform->partition_end; /// If current_row == partitoin_end_row, return true. otherwise - if (current_row != partitoin_end_row) + if (current_row != partition_end_row) { if (current_row.row < transform->blockRowsNumber(current_row)) return false; /// Next row to current_row may belong to next block. - if (partitoin_end_row.block != current_row.block + 1 || partitoin_end_row.row) + if (partition_end_row.block != current_row.block + 1 || partition_end_row.row) return false; } return true; From fa234cadcbd287145ce93211dd239fdf034d9335 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 14 May 2024 11:47:50 +0800 Subject: [PATCH 068/107] update doc --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 814a7ac4aca..16225d4b0e2 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,7 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficient| +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From 297b65dbbe1859bac7c237d644452b03e3e5849d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 15 May 2024 17:16:44 +0800 Subject: [PATCH 069/107] fixed --- src/Processors/Transforms/WindowTransform.cpp | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index ad592613da2..517e202556b 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2267,15 +2267,25 @@ public: bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) - { - LOG_ERROR(getLogger("WindowFunctionPercentRank"), - "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - return false; + if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE + || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) + { + LOG_ERROR( + getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be 'RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT'"); + return false; } return true; } + std::optional getDefaultFrame() const override + { + WindowFrame frame; + frame.type = WindowFrame::FrameType::RANGE; + frame.begin_type = WindowFrame::BoundaryType::Unbounded; + frame.end_type = WindowFrame::BoundaryType::Current; + return frame; + } void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { From b6782d4b2d98e72002b691a3a421d689831fc1bf Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 24 May 2024 09:39:20 +0800 Subject: [PATCH 070/107] update --- src/Processors/Transforms/WindowTransform.cpp | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 517e202556b..729fef5c05d 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1633,24 +1633,24 @@ struct WindowFunctionHelpers ALWAYS_INLINE static bool checkPartitionEnterLastRow(const WindowTransform * transform) { - /// when partition_ended is false, it means that we don't reach the last row in this partition. - /// But when partition_ended is true, it doesn't mean that we reach the last row in this partition. - /// partition_ended is true when - /// - the input has finished. or - /// - current block contains next partition's data. /// This is for fast check. if (!transform->partition_ended) return false; auto current_row = transform->current_row; + /// checkPartitionEnterLastRow is called on each row, also move on current_row.row here. current_row.row++; const auto & partition_end_row = transform->partition_end; - /// If current_row == partitoin_end_row, return true. otherwise + + /// The partition end is reached, when following is true + /// - current row is the partition end row, + /// - or current row is the last row of all input. if (current_row != partition_end_row) { + /// when current row is not the partition end row, we need to check whether it's the last + /// input row. if (current_row.row < transform->blockRowsNumber(current_row)) return false; - /// Next row to current_row may belong to next block. if (partition_end_row.block != current_row.block + 1 || partition_end_row.row) return false; } @@ -2268,7 +2268,8 @@ public: bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE - || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) + || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Current) { LOG_ERROR( getLogger("WindowFunctionPercentRank"), @@ -2305,20 +2306,20 @@ public: } UInt64 remaining_rows = state.current_partition_rows; - Float64 percent_rank_denominator = state.current_partition_rows - 1; + Float64 percent_rank_denominator = remaining_rows == 1 ? 1 : remaining_rows - 1; - if (remaining_rows <= 1) - return; while (remaining_rows > 0) { auto block_rows_number = transform->blockRowsNumber(state.start_row); auto available_block_rows = block_rows_number - state.start_row.row; if (available_block_rows <= remaining_rows) { + /// This partition involves multiple blocks. Finish current block and move on to the + /// next block. auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; auto & data = assert_cast(to_column).getData(); for (size_t i = state.start_row.row; i < block_rows_number; ++i) - data[i] = data[i] / percent_rank_denominator; + data[i] = (data[i] - 1) / percent_rank_denominator; state.start_row.block++; state.start_row.row = 0; @@ -2326,11 +2327,12 @@ public: } else { + /// The partition ends in current block.s auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; auto & data = assert_cast(to_column).getData(); for (size_t i = state.start_row.row, n = state.start_row.row + remaining_rows; i < n; ++i) { - data[i] = data[i]/percent_rank_denominator; + data[i] = (data[i] - 1) / percent_rank_denominator; } state.start_row.row += remaining_rows; remaining_rows = 0; From 87978327d6bab9509c0aef945dfbf76b4437e300 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 May 2024 09:00:17 +0800 Subject: [PATCH 071/107] fixed --- src/Processors/Transforms/WindowTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 729fef5c05d..a694fa43e46 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2350,7 +2350,7 @@ public: inline void insertRankIntoColumn(const WindowTransform * transform, size_t function_index) const { auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; - assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); + assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number)); } }; From bfc755c000cb016f88b51a9526fbb32b375ccbe4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Jul 2024 11:44:24 +0200 Subject: [PATCH 072/107] Fix shutdown --- .../ObjectStorageQueueSource.cpp | 55 +++++++++---------- .../ObjectStorageQueueSource.h | 1 - 2 files changed, 27 insertions(+), 29 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index dc5fb6d2744..f43796fc8b3 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -424,14 +424,14 @@ Chunk ObjectStorageQueueSource::generateImpl() { while (true) { - if (shutdown_called) - { - LOG_TRACE(log, "Shutdown was called, stopping sync"); - break; - } - if (!reader) { + if (shutdown_called) + { + LOG_TEST(log, "Shutdown called"); + break; + } + const auto context = getContext(); reader = StorageObjectStorageSource::createReader( processor_id, file_iterator, configuration, object_storage, read_from_format_info, @@ -448,28 +448,6 @@ Chunk ObjectStorageQueueSource::generateImpl() const auto * object_info = dynamic_cast(reader.getObjectInfo().get()); auto file_metadata = object_info->file_metadata; auto file_status = file_metadata->getFileStatus(); - - if (isCancelled()) - { - reader->cancel(); - - if (processed_rows_from_file) - { - try - { - file_metadata->setFailed("Cancelled", /* reduce_retry_count */true, /* overwrite_status */false); - } - catch (...) - { - LOG_ERROR(log, "Failed to set file {} as failed: {}", - object_info->relative_path, getCurrentExceptionMessage(true)); - } - } - - LOG_TEST(log, "Query is cancelled"); - break; - } - const auto & path = reader.getObjectInfo()->getPath(); if (shutdown_called) @@ -504,6 +482,27 @@ Chunk ObjectStorageQueueSource::generateImpl() path, processed_rows_from_file); } + if (isCancelled()) + { + reader->cancel(); + + if (processed_rows_from_file) + { + try + { + file_metadata->setFailed("Cancelled", /* reduce_retry_count */true, /* overwrite_status */false); + } + catch (...) + { + LOG_ERROR(log, "Failed to set file {} as failed: {}", + object_info->relative_path, getCurrentExceptionMessage(true)); + } + } + + LOG_TEST(log, "Query is cancelled"); + break; + } + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); /// FIXME: if files are compressed, profile counters update does not work fully (object storage related counters are not saved). Why? diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h index fce2a426ecb..0f3d0ab2e92 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h @@ -21,7 +21,6 @@ class ObjectStorageQueueSource : public ISource, WithContext public: using Storage = StorageObjectStorage; using Source = StorageObjectStorageSource; - using RemoveFileFunc = std::function; using BucketHolderPtr = ObjectStorageQueueOrderedFileMetadata::BucketHolderPtr; using BucketHolder = ObjectStorageQueueOrderedFileMetadata::BucketHolder; From 7ae85fda3f0279ceb16a45ee8babdbb0cd40f57d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Jul 2024 11:46:17 +0200 Subject: [PATCH 073/107] Restore previous order --- .../ObjectStorageQueueSource.cpp | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 4f6f6a0e97a..4d921003e04 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -452,6 +452,27 @@ Chunk ObjectStorageQueueSource::generateImpl() auto file_status = file_metadata->getFileStatus(); const auto & path = reader.getObjectInfo()->getPath(); + if (isCancelled()) + { + reader->cancel(); + + if (processed_rows_from_file) + { + try + { + file_metadata->setFailed("Cancelled", /* reduce_retry_count */true, /* overwrite_status */false); + } + catch (...) + { + LOG_ERROR(log, "Failed to set file {} as failed: {}", + object_info->relative_path, getCurrentExceptionMessage(true)); + } + } + + LOG_TEST(log, "Query is cancelled"); + break; + } + if (shutdown_called) { LOG_TEST(log, "Shutdown called"); @@ -484,27 +505,6 @@ Chunk ObjectStorageQueueSource::generateImpl() path, processed_rows_from_file); } - if (isCancelled()) - { - reader->cancel(); - - if (processed_rows_from_file) - { - try - { - file_metadata->setFailed("Cancelled", /* reduce_retry_count */true, /* overwrite_status */false); - } - catch (...) - { - LOG_ERROR(log, "Failed to set file {} as failed: {}", - object_info->relative_path, getCurrentExceptionMessage(true)); - } - } - - LOG_TEST(log, "Query is cancelled"); - break; - } - try { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::ObjectStorageQueuePullMicroseconds); From 33b7afc1b45e0a493f4139dae14025e4e3613c29 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jul 2024 09:02:33 +0000 Subject: [PATCH 074/107] Bump vectorscan to 5.4.11 --- contrib/icu-cmake/CMakeLists.txt | 2 +- contrib/vectorscan | 2 +- contrib/vectorscan-cmake/CMakeLists.txt | 27 +++++--------------- contrib/vectorscan-cmake/common/hs_version.h | 6 ++++- 4 files changed, 14 insertions(+), 23 deletions(-) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index a54bd8c1de2..0a650f2bcc0 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -5,7 +5,7 @@ else () endif () if (NOT ENABLE_ICU) - message(STATUS "Not using icu") + message(STATUS "Not using ICU") return() endif() diff --git a/contrib/vectorscan b/contrib/vectorscan index 4918f81ea3d..d29730e1cb9 160000 --- a/contrib/vectorscan +++ b/contrib/vectorscan @@ -1 +1 @@ -Subproject commit 4918f81ea3d1abd18905bac9876d4a1fe2ebdf07 +Subproject commit d29730e1cb9daaa66bda63426cdce83505d2c809 diff --git a/contrib/vectorscan-cmake/CMakeLists.txt b/contrib/vectorscan-cmake/CMakeLists.txt index d6c626c1612..35d5fd3dc82 100644 --- a/contrib/vectorscan-cmake/CMakeLists.txt +++ b/contrib/vectorscan-cmake/CMakeLists.txt @@ -1,11 +1,8 @@ -# We use vectorscan, a portable and API/ABI-compatible drop-in replacement for hyperscan. - +# Vectorscan is drop-in replacement for Hyperscan. if ((ARCH_AMD64 AND NOT NO_SSE3_OR_HIGHER) OR ARCH_AARCH64) - option (ENABLE_VECTORSCAN "Enable vectorscan library" ${ENABLE_LIBRARIES}) + option (ENABLE_VECTORSCAN "Enable vectorscan" ${ENABLE_LIBRARIES}) endif() -# TODO PPC should generally work but needs manual generation of ppc/config.h file on a PPC machine - if (NOT ENABLE_VECTORSCAN) message (STATUS "Not using vectorscan") return() @@ -272,34 +269,24 @@ if (ARCH_AARCH64) ) endif() -# TODO -# if (ARCH_PPC64LE) -# list(APPEND SRCS -# "${LIBRARY_DIR}/src/util/supervector/arch/ppc64el/impl.cpp" -# ) -# endif() - add_library (_vectorscan ${SRCS}) -target_compile_options (_vectorscan PRIVATE - -fno-sanitize=undefined # assume the library takes care of itself - -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # options from original build system -) # library has too much debug information if (OMIT_HEAVY_DEBUG_SYMBOLS) target_compile_options (_vectorscan PRIVATE -g0) endif() -# Include version header manually generated by running the original build system -target_include_directories (_vectorscan SYSTEM PRIVATE common) +target_include_directories (_vectorscan SYSTEM PUBLIC "${LIBRARY_DIR}/src") + +# Makes the version header visible. It was generated by running the native build system manually. +# Please update whenever you update vectorscan. +target_include_directories (_vectorscan SYSTEM PUBLIC common) # vectorscan inherited some patched in-source versions of boost headers to fix a bug in # boost 1.69. This bug has been solved long ago but vectorscan's source code still # points to the patched versions, so include it here. target_include_directories (_vectorscan SYSTEM PRIVATE "${LIBRARY_DIR}/include") -target_include_directories (_vectorscan SYSTEM PUBLIC "${LIBRARY_DIR}/src") - # Include platform-specific config header generated by manually running the original build system # Please regenerate these files if you update vectorscan. diff --git a/contrib/vectorscan-cmake/common/hs_version.h b/contrib/vectorscan-cmake/common/hs_version.h index 8315b44fb2a..3d266484095 100644 --- a/contrib/vectorscan-cmake/common/hs_version.h +++ b/contrib/vectorscan-cmake/common/hs_version.h @@ -32,8 +32,12 @@ /** * A version string to identify this release of Hyperscan. */ -#define HS_VERSION_STRING "5.4.7 2022-06-20" +#define HS_VERSION_STRING "5.4.11 2024-07-04" #define HS_VERSION_32BIT ((5 << 24) | (1 << 16) | (7 << 8) | 0) +#define HS_MAJOR 5 +#define HS_MINOR 4 +#define HS_PATCH 11 + #endif /* HS_VERSION_H_C6428FAF8E3713 */ From 4543ae3d6490a660c4df8b15bde0345735a540e0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 4 Jul 2024 12:51:22 +0200 Subject: [PATCH 075/107] Update test --- tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql index 95823283812..a01a595dbb5 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql @@ -3,5 +3,5 @@ create table test (d Dynamic) engine=Memory; insert into table test select * from numbers(5); alter table test modify column d Dynamic(max_types=1); select d.UInt64 from test settings allow_experimental_analyzer=1; -select d.UInt64 from test settings allow_experimental_analyzer=1; +select d.UInt64 from test settings allow_experimental_analyzer=0; From 6ea4c101214edf678743833856e09f717f672c67 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 27 Jun 2024 17:19:52 +0200 Subject: [PATCH 076/107] Done --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f6d282792db..9da0b297e9e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -399,7 +399,7 @@ class IColumn; M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ + M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", IMPORTANT) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ From 6f89c4b9328b587d2342ca172da4eaebe0611a1c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 Jun 2024 21:09:36 +0000 Subject: [PATCH 077/107] Bump the minimal version to keep compatibility --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 544b06cca1b..34f5c28fef8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -73,7 +73,7 @@ CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.lo # Minimum version we use in integration tests to check compatibility with old releases # Keep in mind that we only support upgrading between releases that are at most 1 year different. # This means that this minimum need to be, at least, 1 year older than the current release -CLICKHOUSE_CI_MIN_TESTED_VERSION = "22.8" +CLICKHOUSE_CI_MIN_TESTED_VERSION = "23.3" # to create docker-compose env file From d62454714b1e8ea760c6d6baa8df4ec185b8750c Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 1 Jul 2024 17:52:35 +0200 Subject: [PATCH 078/107] Make test error visible --- tests/ci/integration_tests_runner.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 87f721cfde7..7802dfa3c52 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -434,7 +434,14 @@ class ClickhouseIntegrationTestsRunner: "Getting all tests to the file %s with cmd: \n%s", out_file_full, cmd ) with open(out_file_full, "wb") as ofd: - subprocess.check_call(cmd, shell=True, stdout=ofd, stderr=ofd) + try: + subprocess.check_call(cmd, shell=True, stdout=ofd, stderr=ofd) + except subprocess.CalledProcessError as ex: + print("ERROR: Setting test plan failed. Output:") + with open(out_file_full, 'r') as file: + for line in file: + print(" " + line, end='') + raise ex all_tests = set() with open(out_file_full, "r", encoding="utf-8") as all_tests_fd: From e2553454ecbe890cecf90995565d805287ca0703 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Jul 2024 16:02:06 +0000 Subject: [PATCH 079/107] Automatic style fix --- tests/ci/integration_tests_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 7802dfa3c52..a1c33cf22d9 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -438,9 +438,9 @@ class ClickhouseIntegrationTestsRunner: subprocess.check_call(cmd, shell=True, stdout=ofd, stderr=ofd) except subprocess.CalledProcessError as ex: print("ERROR: Setting test plan failed. Output:") - with open(out_file_full, 'r') as file: + with open(out_file_full, "r") as file: for line in file: - print(" " + line, end='') + print(" " + line, end="") raise ex all_tests = set() From 782115efea35a60cd9627faf22c1684ad54a551d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 1 Jul 2024 18:06:47 +0200 Subject: [PATCH 080/107] Very bad change --- tests/integration/test_distributed_inter_server_secret/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 50d7be4d11e..3e656c9d776 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -28,7 +28,7 @@ def make_instance(name, *args, **kwargs): # DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 added in 23.3, ensure that CLICKHOUSE_CI_MIN_TESTED_VERSION fits -assert CLICKHOUSE_CI_MIN_TESTED_VERSION < "23.3" +assert CLICKHOUSE_CI_MIN_TESTED_VERSION <= "23.3" # _n1/_n2 contains cluster with different -- should fail # only n1 contains new_user From 2c37cc048c16a90d972c0f1c2b9c41727174cff3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 1 Jul 2024 22:11:44 +0200 Subject: [PATCH 081/107] Style --- tests/ci/integration_tests_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index a1c33cf22d9..21f16d995a4 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -438,7 +438,7 @@ class ClickhouseIntegrationTestsRunner: subprocess.check_call(cmd, shell=True, stdout=ofd, stderr=ofd) except subprocess.CalledProcessError as ex: print("ERROR: Setting test plan failed. Output:") - with open(out_file_full, "r") as file: + with open(out_file_full, "r", encoding="utf-8") as file: for line in file: print(" " + line, end="") raise ex From 6fcd0eed06af3a02d1e26f182eb7f2bbf16d6471 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Jul 2024 11:40:04 +0000 Subject: [PATCH 082/107] Remove tests which are no longer relevant --- .../test.py | 36 +------------------ 1 file changed, 1 insertion(+), 35 deletions(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 3e656c9d776..457590ac851 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -7,7 +7,7 @@ import uuid import time from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION +from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -26,10 +26,6 @@ def make_instance(name, *args, **kwargs): **kwargs, ) - -# DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 added in 23.3, ensure that CLICKHOUSE_CI_MIN_TESTED_VERSION fits -assert CLICKHOUSE_CI_MIN_TESTED_VERSION <= "23.3" - # _n1/_n2 contains cluster with different -- should fail # only n1 contains new_user n1 = make_instance( @@ -38,14 +34,6 @@ n1 = make_instance( user_configs=["configs/users.d/new_user.xml"], ) n2 = make_instance("n2", main_configs=["configs/remote_servers_n2.xml"]) -backward = make_instance( - "backward", - main_configs=["configs/remote_servers_backward.xml"], - image="clickhouse/clickhouse-server", - # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 - tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, - with_installed_binary=True, -) users = pytest.mark.parametrize( "user,password", @@ -427,28 +415,6 @@ def test_per_user_protocol_settings_secure_cluster(user, password): ) -@users -def test_user_secure_cluster_with_backward(user, password): - id_ = "with-backward-query-dist_secure-" + user - n1.query( - f"SELECT *, '{id_}' FROM dist_secure_backward", user=user, password=password - ) - assert get_query_user_info(n1, id_) == [user, user] - assert get_query_user_info(backward, id_) == [user, user] - - -@users -def test_user_secure_cluster_from_backward(user, password): - id_ = "from-backward-query-dist_secure-" + user - backward.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) - assert get_query_user_info(n1, id_) == [user, user] - assert get_query_user_info(backward, id_) == [user, user] - - assert n1.contains_in_log( - "Using deprecated interserver protocol because the client is too old. Consider upgrading all nodes in cluster." - ) - - def test_secure_cluster_distributed_over_distributed_different_users(): # This works because we will have initial_user='default' n1.query( From 910065e42745e3a1299a596462d6197e4c36a50f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Jul 2024 11:48:28 +0000 Subject: [PATCH 083/107] Automatic style fix --- tests/integration/test_distributed_inter_server_secret/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 457590ac851..7ecb2cda257 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -26,6 +26,7 @@ def make_instance(name, *args, **kwargs): **kwargs, ) + # _n1/_n2 contains cluster with different -- should fail # only n1 contains new_user n1 = make_instance( From dd3eb538f6a0788365ff62e15ab167b28f3d76a1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Jul 2024 22:51:21 +0000 Subject: [PATCH 084/107] Better --- src/Client/HedgedConnections.cpp | 6 ++ src/Client/MultiplexedConnections.cpp | 6 ++ src/Server/TCPHandler.cpp | 7 ++ .../test_analyzer_compatibility/__init__.py | 0 .../configs/remote_servers.xml | 17 ++++ .../test_analyzer_compatibility/test.py | 79 +++++++++++++++++++ 6 files changed, 115 insertions(+) create mode 100644 tests/integration/test_analyzer_compatibility/__init__.py create mode 100644 tests/integration/test_analyzer_compatibility/configs/remote_servers.xml create mode 100644 tests/integration/test_analyzer_compatibility/test.py diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 8c993f906e0..51cbe6f3d6f 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -195,6 +195,12 @@ void HedgedConnections::sendQuery( modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; } + /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. + /// Make the analyzer being set, so it will be effectively applied on the remote server. + /// In other words, the initiator always controls whether the analyzer enabled or not for + /// all servers involved in the distributed query processing. + modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); + replica.connection->sendQuery(timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); replica.packet_receiver->setTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 5d0fc8fd39e..99bdd706d8b 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -150,6 +150,12 @@ void MultiplexedConnections::sendQuery( } } + /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. + /// Make the analyzer being set, so it will be effectively applied on the remote server. + /// In other words, the initiator always controls whether the analyzer enabled or not for + /// all servers involved in the distributed query processing. + modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; size_t num_replicas = replica_states.size(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a522a3f8782..cfb41be0c27 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1890,6 +1890,13 @@ void TCPHandler::receiveQuery() /// /// Settings /// + + /// FIXME: Remove when allow_experimental_analyzer will become obsolete. + /// Even if allow_experimental_analyzer setting wasn't explicitly changed on the initiator server, it might be disabled there + /// So we just force ourselves to act in the same way. + if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + passed_settings.set("allow_experimental_analyzer", static_cast(passed_settings.allow_experimental_analyzer)); + auto settings_changes = passed_settings.changes(); query_kind = query_context->getClientInfo().query_kind; if (query_kind == ClientInfo::QueryKind::INITIAL_QUERY) diff --git a/tests/integration/test_analyzer_compatibility/__init__.py b/tests/integration/test_analyzer_compatibility/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_analyzer_compatibility/configs/remote_servers.xml b/tests/integration/test_analyzer_compatibility/configs/remote_servers.xml new file mode 100644 index 00000000000..0a50dab7fd3 --- /dev/null +++ b/tests/integration/test_analyzer_compatibility/configs/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + true + + current + 9000 + + + backward + 9000 + + + + + diff --git a/tests/integration/test_analyzer_compatibility/test.py b/tests/integration/test_analyzer_compatibility/test.py new file mode 100644 index 00000000000..0ba7f248606 --- /dev/null +++ b/tests/integration/test_analyzer_compatibility/test.py @@ -0,0 +1,79 @@ +import uuid + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +CLICKHOUSE_MAX_VERSION_WITH_ANALYZER_DISABLED_BY_DEFAULT = "24.2" + +cluster = ClickHouseCluster(__file__) +# Here analyzer is enabled by default +current = cluster.add_instance( + "current", + main_configs=["configs/remote_servers.xml"], +) +# Here analyzer is disabled by default +backward = cluster.add_instance( + "backward", + use_old_analyzer=True, + main_configs=["configs/remote_servers.xml"], + image="clickhouse/clickhouse-server", + tag=CLICKHOUSE_MAX_VERSION_WITH_ANALYZER_DISABLED_BY_DEFAULT, + with_installed_binary=True, +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_two_new_versions(start_cluster): + # Two new versions (both know about the analyzer) + # One have it enabled by default, another one - disabled. + + current.query("SYSTEM FLUSH LOGS") + backward.query("SYSTEM FLUSH LOGS") + + query_id = str(uuid.uuid4()) + current.query("SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables);", query_id=query_id) + + current.query("SYSTEM FLUSH LOGS") + backward.query("SYSTEM FLUSH LOGS") + + assert current.query(""" +SELECT hostname() AS h, getSetting('allow_experimental_analyzer') +FROM clusterAllReplicas('test_cluster_mixed', system.one) +ORDER BY h;""") == TSV([["backward", "true"], ["current", "true"]]) + + # Should be enabled everywhere + analyzer_enabled = current.query(f""" +SELECT +DISTINCT Settings['allow_experimental_analyzer'] +FROM clusterAllReplicas('test_cluster_mixed', system.query_log) +WHERE initial_query_id = '{query_id}';""") + + assert TSV(analyzer_enabled) == TSV("1") + + query_id = str(uuid.uuid4()) + backward.query("SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables)", query_id=query_id) + + current.query("SYSTEM FLUSH LOGS") + backward.query("SYSTEM FLUSH LOGS") + + assert backward.query(""" +SELECT hostname() AS h, getSetting('allow_experimental_analyzer') +FROM clusterAllReplicas('test_cluster_mixed', system.one) +ORDER BY h;""") == TSV([["backward", "false"], ["current", "false"]]) + + # Should be disabled everywhere + analyzer_enabled = backward.query(f""" +SELECT +DISTINCT Settings['allow_experimental_analyzer'] +FROM clusterAllReplicas('test_cluster_mixed', system.query_log) +WHERE initial_query_id = '{query_id}';""") + + assert TSV(analyzer_enabled) == TSV("0") From 7bd283764cc7c9350c376c3dc4f4b5c1bba0deee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Jul 2024 22:59:22 +0000 Subject: [PATCH 085/107] Automatic style fix --- .../test_analyzer_compatibility/test.py | 41 ++++++++++++++----- 1 file changed, 31 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_analyzer_compatibility/test.py b/tests/integration/test_analyzer_compatibility/test.py index 0ba7f248606..d4ded420c61 100644 --- a/tests/integration/test_analyzer_compatibility/test.py +++ b/tests/integration/test_analyzer_compatibility/test.py @@ -22,6 +22,7 @@ backward = cluster.add_instance( with_installed_binary=True, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -39,41 +40,61 @@ def test_two_new_versions(start_cluster): backward.query("SYSTEM FLUSH LOGS") query_id = str(uuid.uuid4()) - current.query("SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables);", query_id=query_id) + current.query( + "SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables);", + query_id=query_id, + ) current.query("SYSTEM FLUSH LOGS") backward.query("SYSTEM FLUSH LOGS") - assert current.query(""" + assert ( + current.query( + """ SELECT hostname() AS h, getSetting('allow_experimental_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) -ORDER BY h;""") == TSV([["backward", "true"], ["current", "true"]]) +ORDER BY h;""" + ) + == TSV([["backward", "true"], ["current", "true"]]) + ) # Should be enabled everywhere - analyzer_enabled = current.query(f""" + analyzer_enabled = current.query( + f""" SELECT DISTINCT Settings['allow_experimental_analyzer'] FROM clusterAllReplicas('test_cluster_mixed', system.query_log) -WHERE initial_query_id = '{query_id}';""") +WHERE initial_query_id = '{query_id}';""" + ) assert TSV(analyzer_enabled) == TSV("1") query_id = str(uuid.uuid4()) - backward.query("SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables)", query_id=query_id) + backward.query( + "SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables)", + query_id=query_id, + ) current.query("SYSTEM FLUSH LOGS") backward.query("SYSTEM FLUSH LOGS") - assert backward.query(""" + assert ( + backward.query( + """ SELECT hostname() AS h, getSetting('allow_experimental_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) -ORDER BY h;""") == TSV([["backward", "false"], ["current", "false"]]) +ORDER BY h;""" + ) + == TSV([["backward", "false"], ["current", "false"]]) + ) # Should be disabled everywhere - analyzer_enabled = backward.query(f""" + analyzer_enabled = backward.query( + f""" SELECT DISTINCT Settings['allow_experimental_analyzer'] FROM clusterAllReplicas('test_cluster_mixed', system.query_log) -WHERE initial_query_id = '{query_id}';""") +WHERE initial_query_id = '{query_id}';""" + ) assert TSV(analyzer_enabled) == TSV("0") From d57375181d7de82f28ce5fcd40580cf04c2411b8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Jul 2024 15:49:52 +0000 Subject: [PATCH 086/107] Better --- src/Core/Settings.h | 2 +- tests/integration/helpers/cluster.py | 2 +- .../test.py | 35 ++++++++++++++++++- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9da0b297e9e..f6d282792db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -399,7 +399,7 @@ class IColumn; M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", IMPORTANT) \ + M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 34f5c28fef8..544b06cca1b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -73,7 +73,7 @@ CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.lo # Minimum version we use in integration tests to check compatibility with old releases # Keep in mind that we only support upgrading between releases that are at most 1 year different. # This means that this minimum need to be, at least, 1 year older than the current release -CLICKHOUSE_CI_MIN_TESTED_VERSION = "23.3" +CLICKHOUSE_CI_MIN_TESTED_VERSION = "22.8" # to create docker-compose env file diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 7ecb2cda257..50d7be4d11e 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -7,7 +7,7 @@ import uuid import time from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) @@ -27,6 +27,9 @@ def make_instance(name, *args, **kwargs): ) +# DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 added in 23.3, ensure that CLICKHOUSE_CI_MIN_TESTED_VERSION fits +assert CLICKHOUSE_CI_MIN_TESTED_VERSION < "23.3" + # _n1/_n2 contains cluster with different -- should fail # only n1 contains new_user n1 = make_instance( @@ -35,6 +38,14 @@ n1 = make_instance( user_configs=["configs/users.d/new_user.xml"], ) n2 = make_instance("n2", main_configs=["configs/remote_servers_n2.xml"]) +backward = make_instance( + "backward", + main_configs=["configs/remote_servers_backward.xml"], + image="clickhouse/clickhouse-server", + # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 + tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, + with_installed_binary=True, +) users = pytest.mark.parametrize( "user,password", @@ -416,6 +427,28 @@ def test_per_user_protocol_settings_secure_cluster(user, password): ) +@users +def test_user_secure_cluster_with_backward(user, password): + id_ = "with-backward-query-dist_secure-" + user + n1.query( + f"SELECT *, '{id_}' FROM dist_secure_backward", user=user, password=password + ) + assert get_query_user_info(n1, id_) == [user, user] + assert get_query_user_info(backward, id_) == [user, user] + + +@users +def test_user_secure_cluster_from_backward(user, password): + id_ = "from-backward-query-dist_secure-" + user + backward.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) + assert get_query_user_info(n1, id_) == [user, user] + assert get_query_user_info(backward, id_) == [user, user] + + assert n1.contains_in_log( + "Using deprecated interserver protocol because the client is too old. Consider upgrading all nodes in cluster." + ) + + def test_secure_cluster_distributed_over_distributed_different_users(): # This works because we will have initial_user='default' n1.query( From fcabefa8f3e5a86aad6f5c2b79ef8eabbc349b9d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Jul 2024 16:42:01 +0000 Subject: [PATCH 087/107] Automatically disabling --- src/Server/TCPHandler.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cfb41be0c27..443cc99475f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1892,10 +1892,13 @@ void TCPHandler::receiveQuery() /// /// FIXME: Remove when allow_experimental_analyzer will become obsolete. - /// Even if allow_experimental_analyzer setting wasn't explicitly changed on the initiator server, it might be disabled there - /// So we just force ourselves to act in the same way. - if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - passed_settings.set("allow_experimental_analyzer", static_cast(passed_settings.allow_experimental_analyzer)); + /// Analyzer became Beta in 24.3 and started to be enabled by default. + /// We have to disable it for ourselves to make sure we don't have different settings on + /// different servers. + if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY + && client_info.getVersionNumber() < VersionNumber(23, 3, 0) + && !passed_settings.allow_experimental_analyzer.changed) + passed_settings.set("allow_experimental_analyzer", false); auto settings_changes = passed_settings.changes(); query_kind = query_context->getClientInfo().query_kind; From fe6a875c7473d814011f4ae202942232e0801427 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Jul 2024 21:51:40 +0000 Subject: [PATCH 088/107] Make the setting back IMPORTANT + fix build --- src/Core/Settings.h | 2 +- src/Server/TCPHandler.cpp | 2 +- tests/integration/helpers/cluster.py | 2 +- .../test.py | 35 +------------------ 4 files changed, 4 insertions(+), 37 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f6d282792db..9da0b297e9e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -399,7 +399,7 @@ class IColumn; M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ + M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", IMPORTANT) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 443cc99475f..ac1423f87c1 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1875,7 +1875,7 @@ void TCPHandler::receiveQuery() #endif } - query_context = session->makeQueryContext(std::move(client_info)); + query_context = session->makeQueryContext(client_info); /// Sets the default database if it wasn't set earlier for the session context. if (is_interserver_mode && !default_database.empty()) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 544b06cca1b..34f5c28fef8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -73,7 +73,7 @@ CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.lo # Minimum version we use in integration tests to check compatibility with old releases # Keep in mind that we only support upgrading between releases that are at most 1 year different. # This means that this minimum need to be, at least, 1 year older than the current release -CLICKHOUSE_CI_MIN_TESTED_VERSION = "22.8" +CLICKHOUSE_CI_MIN_TESTED_VERSION = "23.3" # to create docker-compose env file diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 50d7be4d11e..7ecb2cda257 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -7,7 +7,7 @@ import uuid import time from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION +from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -27,9 +27,6 @@ def make_instance(name, *args, **kwargs): ) -# DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 added in 23.3, ensure that CLICKHOUSE_CI_MIN_TESTED_VERSION fits -assert CLICKHOUSE_CI_MIN_TESTED_VERSION < "23.3" - # _n1/_n2 contains cluster with different -- should fail # only n1 contains new_user n1 = make_instance( @@ -38,14 +35,6 @@ n1 = make_instance( user_configs=["configs/users.d/new_user.xml"], ) n2 = make_instance("n2", main_configs=["configs/remote_servers_n2.xml"]) -backward = make_instance( - "backward", - main_configs=["configs/remote_servers_backward.xml"], - image="clickhouse/clickhouse-server", - # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 - tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, - with_installed_binary=True, -) users = pytest.mark.parametrize( "user,password", @@ -427,28 +416,6 @@ def test_per_user_protocol_settings_secure_cluster(user, password): ) -@users -def test_user_secure_cluster_with_backward(user, password): - id_ = "with-backward-query-dist_secure-" + user - n1.query( - f"SELECT *, '{id_}' FROM dist_secure_backward", user=user, password=password - ) - assert get_query_user_info(n1, id_) == [user, user] - assert get_query_user_info(backward, id_) == [user, user] - - -@users -def test_user_secure_cluster_from_backward(user, password): - id_ = "from-backward-query-dist_secure-" + user - backward.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) - assert get_query_user_info(n1, id_) == [user, user] - assert get_query_user_info(backward, id_) == [user, user] - - assert n1.contains_in_log( - "Using deprecated interserver protocol because the client is too old. Consider upgrading all nodes in cluster." - ) - - def test_secure_cluster_distributed_over_distributed_different_users(): # This works because we will have initial_user='default' n1.query( From c93d8cbb66ad7ae5a1e9c4f46f0c351944ff04e9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Jul 2024 13:57:47 +0200 Subject: [PATCH 089/107] Fixes --- contrib/jemalloc-cmake/CMakeLists.txt | 6 +++++- programs/keeper/Keeper.cpp | 5 +++++ programs/server/Server.cpp | 5 +++++ src/Common/Jemalloc.cpp | 26 +++++++++++++++++++++++++- src/Common/Jemalloc.h | 4 ++++ 5 files changed, 44 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index cc5a391676f..38ebcc8f680 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -34,7 +34,11 @@ if (OS_LINUX) # avoid spurious latencies and additional work associated with # MADV_DONTNEED. See # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000,prof:true,prof_active:false") + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000") + else() + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000,prof:true,prof_active:false,background_thread:true") + endif() else() set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000") endif() diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index f14ef2e5552..fc3778593a6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -27,6 +27,8 @@ #include #include +#include + #include #include @@ -277,6 +279,9 @@ HTTPContextPtr httpContext() int Keeper::main(const std::vector & /*args*/) try { +#if USE_JEMALLOC + setJemallocBackgroundThreads(true); +#endif Poco::Logger * log = &logger(); UseSSL use_ssl; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4cb3b5f45c7..1277249b462 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -656,6 +657,10 @@ static void initializeAzureSDKLogger( int Server::main(const std::vector & /*args*/) try { +#if USE_JEMALLOC + setJemallocBackgroundThreads(true); +#endif + Stopwatch startup_watch; Poco::Logger * log = &logger(); diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index fbe2f62c944..d7cc246db6a 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -46,6 +46,20 @@ void checkJemallocProfilingEnabled() "set: MALLOC_CONF=background_thread:true,prof:true"); } +template +void setJemallocValue(const char * name, T value) +{ + T old_value; + size_t old_value_size = sizeof(T); + if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) + { + LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); + return; + } + + LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); +} + void setJemallocProfileActive(bool value) { checkJemallocProfilingEnabled(); @@ -58,7 +72,7 @@ void setJemallocProfileActive(bool value) return; } - mallctl("prof.active", nullptr, nullptr, &value, sizeof(bool)); + setJemallocValue("prof.active", value); LOG_TRACE(getLogger("SystemJemalloc"), "Profiling is {}", value ? "enabled" : "disabled"); } @@ -84,6 +98,16 @@ std::string flushJemallocProfile(const std::string & file_prefix) return profile_dump_path; } +void setJemallocBackgroundThreads(bool enabled) +{ + setJemallocValue("background_thread", enabled); +} + +void setJemallocMaxBackgroundThreads(size_t max_threads) +{ + setJemallocValue("max_background_threads", max_threads); +} + } #endif diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 80ff0f1a319..499a906fd3d 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -17,6 +17,10 @@ void setJemallocProfileActive(bool value); std::string flushJemallocProfile(const std::string & file_prefix); +void setJemallocBackgroundThreads(bool enabled); + +void setJemallocMaxBackgroundThreads(size_t max_threads); + } #endif From da5f3c1efd940488413b34d3e5f8855460f0ce80 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 4 Jul 2024 13:01:57 +0000 Subject: [PATCH 090/107] Fix test --- .../test_functions.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 758dda655da..e5023c062ff 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -130,10 +130,13 @@ def test_string_functions(start_cluster): functions = map(lambda x: x.strip(), functions) excludes = [ + # The argument of this function is not a seed, but an arbitrary expression needed for bypassing common subexpression elimination. "rand", "rand64", "randConstant", + "randCanonical", "generateUUIDv4", + "generateULID", # Syntax error otherwise "position", "substring", @@ -153,6 +156,16 @@ def test_string_functions(start_cluster): "tryBase64Decode", # Removed in 23.9 "meiliMatch", + # These functions require more than one argument. + "parseDateTimeInJodaSyntaxOrZero", + "parseDateTimeInJodaSyntaxOrNull", + "parseDateTimeOrNull", + "parseDateTimeOrZero", + "parseDateTime", + # The argument is effectively a disk name (and we don't have one with name foo) + "filesystemUnreserved", + "filesystemCapacity", + "filesystemAvailable", ] functions = filter(lambda x: x not in excludes, functions) @@ -205,6 +218,9 @@ def test_string_functions(start_cluster): # Function X takes exactly one parameter: # The function 'X' can only be used as a window function "BAD_ARGUMENTS", + # String foo is obviously not a valid IP address. + "CANNOT_PARSE_IPV4", + "CANNOT_PARSE_IPV6", ] if any(map(lambda x: x in error_message, allowed_errors)): logging.info("Skipping %s", function) From dbfe6323821739af4c8856e799cb5d861377439d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 4 Jul 2024 13:11:13 +0000 Subject: [PATCH 091/107] Correct the test to exclude farmHash for now --- tests/integration/test_backward_compatibility/test_functions.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index e5023c062ff..fc03a77030e 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -166,6 +166,8 @@ def test_string_functions(start_cluster): "filesystemUnreserved", "filesystemCapacity", "filesystemAvailable", + # Exclude it for now. Looks like the result depends on the build type. + "farmHash64", ] functions = filter(lambda x: x not in excludes, functions) From 32a61e6088a8d5b18d7f217c45c27f1268db2501 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 15:38:44 +0200 Subject: [PATCH 092/107] Bump From 1e5bc5bc8c7e65908e7127e06e1f5008f6fbfa20 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Jul 2024 15:48:12 +0200 Subject: [PATCH 093/107] fix flaky test --- .../0_stateless/03172_error_log_table_not_empty.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03172_error_log_table_not_empty.sh b/tests/queries/0_stateless/03172_error_log_table_not_empty.sh index 8d74ebe1039..4b83400f5de 100755 --- a/tests/queries/0_stateless/03172_error_log_table_not_empty.sh +++ b/tests/queries/0_stateless/03172_error_log_table_not_empty.sh @@ -4,17 +4,19 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# system.error_log is created lazy, flush logs query makes it sure that the table is created. +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS;" + # Get the previous number of errors for 111, 222 and 333 errors_111=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHERE code = 111") errors_222=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHERE code = 222") errors_333=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHERE code = 333") -# Throw three random errors: 111, 222 and 333 and wait for more than collect_interval_milliseconds to ensure system.error_log is flushed +# Throw three random errors: 111, 222 and 333 and call flush logs to ensure system.error_log is flushed $CLICKHOUSE_CLIENT -mn -q " SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } -SELECT sleep(2) format NULL; SYSTEM FLUSH LOGS; " @@ -30,7 +32,6 @@ $CLICKHOUSE_CLIENT -mn -q " SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } -SELECT sleep(2) format NULL; SYSTEM FLUSH LOGS; " @@ -38,4 +39,4 @@ $CLICKHOUSE_CLIENT -mn -q " SELECT sum(value) > $(($errors_111+1)) FROM system.error_log WHERE code = 111; SELECT sum(value) > $(($errors_222+1)) FROM system.error_log WHERE code = 222; SELECT sum(value) > $(($errors_333+1)) FROM system.error_log WHERE code = 333; -" \ No newline at end of file +" From 11a30d6d6026dca61247590ce337e8e766601c5a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jul 2024 14:14:19 +0000 Subject: [PATCH 094/107] Bump s2geometry to latest master --- contrib/s2geometry | 2 +- contrib/s2geometry-cmake/CMakeLists.txt | 19 ++++++++++++++----- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/contrib/s2geometry b/contrib/s2geometry index 0547c383717..0146e2d1355 160000 --- a/contrib/s2geometry +++ b/contrib/s2geometry @@ -1 +1 @@ -Subproject commit 0547c38371777a1c1c8be263a6f05c3bf71bb05b +Subproject commit 0146e2d1355828f8f633cb050948250ad7406c57 diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 6632f9c27d5..48562b8cead 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ -option(ENABLE_S2_GEOMETRY "Enable S2 geometry library" ${ENABLE_LIBRARIES}) +option(ENABLE_S2_GEOMETRY "Enable S2 Geometry" ${ENABLE_LIBRARIES}) if (NOT ENABLE_S2_GEOMETRY) - message(STATUS "Not using S2 geometry") + message(STATUS "Not using S2 Geometry") return() endif() @@ -38,6 +38,7 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2cell_index.cc" "${S2_SOURCE_DIR}/s2/s2cell_union.cc" "${S2_SOURCE_DIR}/s2/s2centroids.cc" + "${S2_SOURCE_DIR}/s2/s2chain_interpolation_query.cc" "${S2_SOURCE_DIR}/s2/s2closest_cell_query.cc" "${S2_SOURCE_DIR}/s2/s2closest_edge_query.cc" "${S2_SOURCE_DIR}/s2/s2closest_point_query.cc" @@ -46,6 +47,7 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2coords.cc" "${S2_SOURCE_DIR}/s2/s2crossing_edge_query.cc" "${S2_SOURCE_DIR}/s2/s2debug.cc" + "${S2_SOURCE_DIR}/s2/s2density_tree.cc" "${S2_SOURCE_DIR}/s2/s2earth.cc" "${S2_SOURCE_DIR}/s2/s2edge_clipping.cc" "${S2_SOURCE_DIR}/s2/s2edge_crosser.cc" @@ -53,8 +55,10 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2edge_distances.cc" "${S2_SOURCE_DIR}/s2/s2edge_tessellator.cc" "${S2_SOURCE_DIR}/s2/s2error.cc" + "${S2_SOURCE_DIR}/s2/s2fractal.cc" "${S2_SOURCE_DIR}/s2/s2furthest_edge_query.cc" "${S2_SOURCE_DIR}/s2/s2hausdorff_distance_query.cc" + "${S2_SOURCE_DIR}/s2/s2index_cell_data.cc" "${S2_SOURCE_DIR}/s2/s2latlng.cc" "${S2_SOURCE_DIR}/s2/s2latlng_rect.cc" "${S2_SOURCE_DIR}/s2/s2latlng_rect_bounder.cc" @@ -63,10 +67,10 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2lax_polyline_shape.cc" "${S2_SOURCE_DIR}/s2/s2loop.cc" "${S2_SOURCE_DIR}/s2/s2loop_measures.cc" + "${S2_SOURCE_DIR}/s2/s2max_distance_targets.cc" "${S2_SOURCE_DIR}/s2/s2measures.cc" "${S2_SOURCE_DIR}/s2/s2memory_tracker.cc" "${S2_SOURCE_DIR}/s2/s2metrics.cc" - "${S2_SOURCE_DIR}/s2/s2max_distance_targets.cc" "${S2_SOURCE_DIR}/s2/s2min_distance_targets.cc" "${S2_SOURCE_DIR}/s2/s2padded_cell.cc" "${S2_SOURCE_DIR}/s2/s2point_compression.cc" @@ -80,10 +84,11 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2predicates.cc" "${S2_SOURCE_DIR}/s2/s2projections.cc" "${S2_SOURCE_DIR}/s2/s2r2rect.cc" - "${S2_SOURCE_DIR}/s2/s2region.cc" - "${S2_SOURCE_DIR}/s2/s2region_term_indexer.cc" + "${S2_SOURCE_DIR}/s2/s2random.cc" "${S2_SOURCE_DIR}/s2/s2region_coverer.cc" "${S2_SOURCE_DIR}/s2/s2region_intersection.cc" + "${S2_SOURCE_DIR}/s2/s2region_sharder.cc" + "${S2_SOURCE_DIR}/s2/s2region_term_indexer.cc" "${S2_SOURCE_DIR}/s2/s2region_union.cc" "${S2_SOURCE_DIR}/s2/s2shape_index.cc" "${S2_SOURCE_DIR}/s2/s2shape_index_buffered_region.cc" @@ -94,9 +99,12 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2shapeutil_coding.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_contains_brute_force.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_conversion.cc" + "${S2_SOURCE_DIR}/s2/s2shapeutil_count_vertices.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_edge_iterator.cc" + "${S2_SOURCE_DIR}/s2/s2shapeutil_edge_wrap.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_get_reference_point.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_visit_crossing_edge_pairs.cc" + "${S2_SOURCE_DIR}/s2/s2testing.cc" "${S2_SOURCE_DIR}/s2/s2text_format.cc" "${S2_SOURCE_DIR}/s2/s2wedge_relations.cc" "${S2_SOURCE_DIR}/s2/s2winding_operation.cc" @@ -140,6 +148,7 @@ target_link_libraries(_s2 PRIVATE absl::strings absl::type_traits absl::utility + absl::vlog_is_on ) target_include_directories(_s2 SYSTEM BEFORE PUBLIC "${S2_SOURCE_DIR}/") From e7e2b0953c0df57c95f6b5e0f6cc1afe5914c4e9 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 4 Jul 2024 14:50:51 +0000 Subject: [PATCH 095/107] Prevent another possible buffer overflow --- src/Functions/bitShiftLeft.cpp | 4 ++-- src/Functions/bitShiftRight.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 8e39ed86461..d561430d51f 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -42,7 +42,7 @@ struct BitShiftLeftImpl { const UInt8 word_size = 8 * sizeof(*pos); size_t n = end - pos; - const UInt256 bit_limit = word_size * n; + const UInt128 bit_limit = static_cast(word_size) * n; if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if (b == bit_limit) @@ -110,7 +110,7 @@ struct BitShiftLeftImpl { const UInt8 word_size = 8; size_t n = end - pos; - const UInt256 bit_limit = word_size * n; + const UInt128 bit_limit = static_cast(word_size) * n; if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if (b == bit_limit) diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 46cfcde8a33..05b8581c792 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -58,7 +58,7 @@ struct BitShiftRightImpl { const UInt8 word_size = 8; size_t n = end - pos; - const UInt256 bit_limit = word_size * n; + const UInt128 bit_limit = static_cast(word_size) * n; if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if (b == bit_limit) @@ -98,7 +98,7 @@ struct BitShiftRightImpl { const UInt8 word_size = 8; size_t n = end - pos; - const UInt256 bit_limit = word_size * n; + const UInt128 bit_limit = static_cast(word_size) * n; if (b < 0 || static_cast(b) > bit_limit) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); else if (b == bit_limit) From 24ff0f601d5b8d474429d67b5ed8702c662c58ec Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 4 Jul 2024 17:15:32 +0200 Subject: [PATCH 096/107] update keeper bench example config file --- utils/keeper-bench/example.yaml | 67 +++++++++++++++++---------------- 1 file changed, 34 insertions(+), 33 deletions(-) diff --git a/utils/keeper-bench/example.yaml b/utils/keeper-bench/example.yaml index e800e923482..c3a62a01eac 100644 --- a/utils/keeper-bench/example.yaml +++ b/utils/keeper-bench/example.yaml @@ -18,45 +18,46 @@ connections: host: "localhost:9181" -generator: - setup: +setup: + node: + name: "test3" + node: + name: "test_create" + node: + name: "test4" + node: + name: "test" + data: "somedata" node: - name: "test3" + repeat: 4 + name: + random_string: + size: 15 + data: + random_string: + size: + min_value: 10 + max_value: 20 node: - name: "test_create" - node: - name: "test4" - node: - name: "test" - data: "somedata" - node: - repeat: 4 - name: - random_string: - size: 15 - data: - random_string: - size: - min_value: 10 - max_value: 20 + repeat: 2 node: repeat: 2 - node: - repeat: 2 - name: - random_string: - size: 12 name: random_string: - size: 15 - data: - random_string: - size: - min_value: 10 - max_value: 20 - node: - name: "test2" - data: "somedata" + size: 12 + name: + random_string: + size: 15 + data: + random_string: + size: + min_value: 10 + max_value: 20 + node: + name: "test2" + data: "somedata" + +generator: requests: create: path: "/test_create" From 2c9421812063cc22133f508e10033accb611d6d1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jul 2024 15:01:40 +0000 Subject: [PATCH 097/107] Random header fixes for libcxx 16 --- base/poco/Foundation/include/Poco/Logger.h | 2 ++ base/poco/Foundation/include/Poco/Message.h | 1 + src/Common/formatIPv6.h | 1 + src/Coordination/Changelog.h | 1 + src/Coordination/FourLetterCommand.h | 1 + src/Disks/ObjectStorages/MetadataOperationsHolder.h | 1 + src/IO/Archives/hasRegisteredArchiveFileExtension.cpp | 2 ++ src/Loggers/OwnSplitChannel.h | 1 + src/Storages/MergeTree/IPartMetadataManager.h | 1 + 9 files changed, 11 insertions(+) diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index 2a1cb33b407..74ddceea9dd 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -21,6 +21,8 @@ #include #include #include +#include +#include #include #include "Poco/Channel.h" diff --git a/base/poco/Foundation/include/Poco/Message.h b/base/poco/Foundation/include/Poco/Message.h index 9068e56a93c..756e427c5f5 100644 --- a/base/poco/Foundation/include/Poco/Message.h +++ b/base/poco/Foundation/include/Poco/Message.h @@ -19,6 +19,7 @@ #include +#include #include "Poco/Foundation.h" #include "Poco/Timestamp.h" diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index bb83e0381ef..abeda95ed0d 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 2e8dbe75e90..c9b45d9a344 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -5,6 +5,7 @@ #include #include +#include #include #include diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 82b30a0b5f6..2a53bade62f 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -2,6 +2,7 @@ #include "config.h" +#include #include #include #include diff --git a/src/Disks/ObjectStorages/MetadataOperationsHolder.h b/src/Disks/ObjectStorages/MetadataOperationsHolder.h index 8997f40b9a2..a042f4bd8b9 100644 --- a/src/Disks/ObjectStorages/MetadataOperationsHolder.h +++ b/src/Disks/ObjectStorages/MetadataOperationsHolder.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp index 2a979f500f7..407977f1f13 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -1,5 +1,7 @@ #include +#include +#include namespace DB { diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 7ca27cf6584..88bb6b9ce76 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h index cef1d10e4ad..e817421f7d0 100644 --- a/src/Storages/MergeTree/IPartMetadataManager.h +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include From 6dd13dd34ab397d5e18d01820a064f18ed25595a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 4 Jul 2024 17:59:07 +0200 Subject: [PATCH 098/107] fix clean-up process --- utils/keeper-bench/Runner.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 5ae4c7a0b1c..587e015b340 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1311,9 +1311,9 @@ void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & pa while (!children_span.empty()) { Coordination::Requests ops; - for (size_t i = 0; i < 1000 && !children.empty(); ++i) + for (size_t i = 0; i < 1000 && !children_span.empty(); ++i) { - removeRecursive(zookeeper, fs::path(path) / children.back()); + removeRecursive(zookeeper, fs::path(path) / children_span.back()); ops.emplace_back(zkutil::makeRemoveRequest(fs::path(path) / children_span.back(), -1)); children_span = children_span.subspan(0, children_span.size() - 1); } From 78a2139f2a43752196a029995b6965ada359c954 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Jul 2024 19:27:10 +0200 Subject: [PATCH 099/107] restore timeouts, mark as no-fasttests --- .../queries/0_stateless/03172_error_log_table_not_empty.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03172_error_log_table_not_empty.sh b/tests/queries/0_stateless/03172_error_log_table_not_empty.sh index 4b83400f5de..22a2fd82c64 100755 --- a/tests/queries/0_stateless/03172_error_log_table_not_empty.sh +++ b/tests/queries/0_stateless/03172_error_log_table_not_empty.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: this test relies on the timeouts, it always takes no less that 4 seconds to run CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,11 +14,12 @@ errors_111=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHER errors_222=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHERE code = 222") errors_333=$($CLICKHOUSE_CLIENT -q "SELECT sum(value) FROM system.error_log WHERE code = 333") -# Throw three random errors: 111, 222 and 333 and call flush logs to ensure system.error_log is flushed +# Throw three random errors: 111, 222 and 333 and wait for more than collect_interval_milliseconds to ensure system.error_log is flushed $CLICKHOUSE_CLIENT -mn -q " SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } +SELECT sleep(2) format NULL; SYSTEM FLUSH LOGS; " @@ -32,6 +35,7 @@ $CLICKHOUSE_CLIENT -mn -q " SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } +SELECT sleep(2) format NULL; SYSTEM FLUSH LOGS; " From df0cce24ee4f7e74b53c667cfc8b43a7e3e142ea Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 4 Jul 2024 19:34:47 +0200 Subject: [PATCH 100/107] CI: Fix sync pr merge --- tests/ci/sync_pr.py | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index 8251ccbaf38..1b71231f820 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -101,23 +101,20 @@ def main(): assert pr_info.merged_pr, "BUG. merged PR number could not been determined" prs = gh.get_pulls_from_search( - query=f"head:sync-upstream/pr/{pr_info.merged_pr} org:ClickHouse type:pr", + query=f"head:sync-upstream/pr/{pr_info.merged_pr} org:ClickHouse type:pr is:open", repo="ClickHouse/clickhouse-private", ) - sync_pr = None - if len(prs) > 1: print(f"WARNING: More than one PR found [{prs}] - exiting") elif len(prs) == 0: print("WARNING: No Sync PR found") else: sync_pr = prs[0] - - if args.merge: - merge_sync_pr(gh, sync_pr) - elif args.status: - set_sync_status(gh, pr_info, sync_pr) + if args.merge: + merge_sync_pr(gh, sync_pr) + elif args.status: + set_sync_status(gh, pr_info, sync_pr) if __name__ == "__main__": From 597810b69d7655b9049ded3b529d12a23996a770 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jul 2024 18:46:09 +0000 Subject: [PATCH 101/107] Fix s390x build --- contrib/s2geometry-cmake/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 48562b8cead..5eabe71b538 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -1,6 +1,7 @@ option(ENABLE_S2_GEOMETRY "Enable S2 Geometry" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_S2_GEOMETRY) +# ARCH_S390X broke upstream, it can be re-enabled once https://github.com/google/s2geometry/pull/372 is merged +if (NOT ENABLE_S2_GEOMETRY OR ARCH_S390X) message(STATUS "Not using S2 Geometry") return() endif() From ffe1f8fea019f08de4a9a32f99c1ebce4baeae71 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jul 2024 15:40:19 +0000 Subject: [PATCH 102/107] Bump Azure to 1.12 --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 6262a76ef4c..92c94d7f37a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 6262a76ef4c4c330c84e58dd4f6f13f4e6230fcd +Subproject commit 92c94d7f37a43cc8fc4d466884a95f610c0593bf From e428542b2ea7340d6314d76c6043134c356677a0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 18 Feb 2024 12:44:52 +0100 Subject: [PATCH 103/107] Add prometheus protobufs. --- contrib/CMakeLists.txt | 2 + .../prometheus-protobufs-cmake/CMakeLists.txt | 34 +++ contrib/prometheus-protobufs-gogo/LICENSE | 35 +++ contrib/prometheus-protobufs-gogo/README | 4 + .../gogoproto/gogo.proto | 145 +++++++++++++ contrib/prometheus-protobufs/LICENSE | 201 ++++++++++++++++++ contrib/prometheus-protobufs/README | 2 + .../prometheus-protobufs/prompb/remote.proto | 88 ++++++++ .../prometheus-protobufs/prompb/types.proto | 187 ++++++++++++++++ src/CMakeLists.txt | 4 + src/Common/config.h.in | 1 + src/configure_config.cmake | 3 + 12 files changed, 706 insertions(+) create mode 100644 contrib/prometheus-protobufs-cmake/CMakeLists.txt create mode 100644 contrib/prometheus-protobufs-gogo/LICENSE create mode 100644 contrib/prometheus-protobufs-gogo/README create mode 100644 contrib/prometheus-protobufs-gogo/gogoproto/gogo.proto create mode 100644 contrib/prometheus-protobufs/LICENSE create mode 100644 contrib/prometheus-protobufs/README create mode 100644 contrib/prometheus-protobufs/prompb/remote.proto create mode 100644 contrib/prometheus-protobufs/prompb/types.proto diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 08f58335d16..90ae5981a21 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -228,6 +228,8 @@ add_contrib (ulid-c-cmake ulid-c) add_contrib (libssh-cmake libssh) +add_contrib (prometheus-protobufs-cmake prometheus-protobufs prometheus-protobufs-gogo) + # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/prometheus-protobufs-cmake/CMakeLists.txt b/contrib/prometheus-protobufs-cmake/CMakeLists.txt new file mode 100644 index 00000000000..8c939902be7 --- /dev/null +++ b/contrib/prometheus-protobufs-cmake/CMakeLists.txt @@ -0,0 +1,34 @@ +option(ENABLE_PROMETHEUS_PROTOBUFS "Enable Prometheus Protobufs" ${ENABLE_PROTOBUF}) + +if(NOT ENABLE_PROMETHEUS_PROTOBUFS) + message(STATUS "Not using prometheus-protobufs") + return() +endif() + +set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src") +set(Prometheus_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/prometheus-protobufs") +set(GogoProto_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/prometheus-protobufs-gogo") + +# Protobuf_IMPORT_DIRS specify where the protobuf compiler will look for .proto files. +set(Old_Protobuf_IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) +list(APPEND Protobuf_IMPORT_DIRS "${Protobuf_INCLUDE_DIR}" "${Prometheus_INCLUDE_DIR}" "${GogoProto_INCLUDE_DIR}") + +PROTOBUF_GENERATE_CPP(prometheus_protobufs_sources prometheus_protobufs_headers + "prompb/remote.proto" + "prompb/types.proto" + "gogoproto/gogo.proto" +) + +set(Protobuf_IMPORT_DIRS ${Old_Protobuf_IMPORT_DIRS}) + +# Ignore warnings while compiling protobuf-generated *.pb.h and *.pb.cpp files. +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + +# Disable clang-tidy for protobuf-generated *.pb.h and *.pb.cpp files. +set (CMAKE_CXX_CLANG_TIDY "") + +add_library(_prometheus_protobufs ${prometheus_protobufs_sources} ${prometheus_protobufs_headers}) +target_include_directories(_prometheus_protobufs SYSTEM PUBLIC "${CMAKE_CURRENT_BINARY_DIR}") +target_link_libraries (_prometheus_protobufs PUBLIC ch_contrib::protobuf) + +add_library (ch_contrib::prometheus_protobufs ALIAS _prometheus_protobufs) diff --git a/contrib/prometheus-protobufs-gogo/LICENSE b/contrib/prometheus-protobufs-gogo/LICENSE new file mode 100644 index 00000000000..16be18e5c50 --- /dev/null +++ b/contrib/prometheus-protobufs-gogo/LICENSE @@ -0,0 +1,35 @@ +Copyright (c) 2022, The Cosmos SDK Authors. All rights reserved. +Copyright (c) 2013, The GoGo Authors. All rights reserved. + +Protocol Buffers for Go with Gadgets + +Go support for Protocol Buffers - Google's data interchange format + +Copyright 2010 The Go Authors. All rights reserved. +https://github.com/golang/protobuf + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/prometheus-protobufs-gogo/README b/contrib/prometheus-protobufs-gogo/README new file mode 100644 index 00000000000..c40bc42df66 --- /dev/null +++ b/contrib/prometheus-protobufs-gogo/README @@ -0,0 +1,4 @@ +File "gogoproto/gogo.proto" was downloaded from the "Protocol Buffers for Go with Gadgets" project: +https://github.com/cosmos/gogoproto/blob/main/gogoproto/gogo.proto + +File "gogoproto/gogo.proto" is used in ClickHouse to compile prometheus protobufs. diff --git a/contrib/prometheus-protobufs-gogo/gogoproto/gogo.proto b/contrib/prometheus-protobufs-gogo/gogoproto/gogo.proto new file mode 100644 index 00000000000..974b36a7ccd --- /dev/null +++ b/contrib/prometheus-protobufs-gogo/gogoproto/gogo.proto @@ -0,0 +1,145 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/cosmos/gogoproto +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +syntax = "proto2"; +package gogoproto; + +import "google/protobuf/descriptor.proto"; + +option java_package = "com.google.protobuf"; +option java_outer_classname = "GoGoProtos"; +option go_package = "github.com/cosmos/gogoproto/gogoproto"; + +extend google.protobuf.EnumOptions { + optional bool goproto_enum_prefix = 62001; + optional bool goproto_enum_stringer = 62021; + optional bool enum_stringer = 62022; + optional string enum_customname = 62023; + optional bool enumdecl = 62024; +} + +extend google.protobuf.EnumValueOptions { + optional string enumvalue_customname = 66001; +} + +extend google.protobuf.FileOptions { + optional bool goproto_getters_all = 63001; + optional bool goproto_enum_prefix_all = 63002; + optional bool goproto_stringer_all = 63003; + optional bool verbose_equal_all = 63004; + optional bool face_all = 63005; + optional bool gostring_all = 63006; + optional bool populate_all = 63007; + optional bool stringer_all = 63008; + optional bool onlyone_all = 63009; + + optional bool equal_all = 63013; + optional bool description_all = 63014; + optional bool testgen_all = 63015; + optional bool benchgen_all = 63016; + optional bool marshaler_all = 63017; + optional bool unmarshaler_all = 63018; + optional bool stable_marshaler_all = 63019; + + optional bool sizer_all = 63020; + + optional bool goproto_enum_stringer_all = 63021; + optional bool enum_stringer_all = 63022; + + optional bool unsafe_marshaler_all = 63023; + optional bool unsafe_unmarshaler_all = 63024; + + optional bool goproto_extensions_map_all = 63025; + optional bool goproto_unrecognized_all = 63026; + optional bool gogoproto_import = 63027; + optional bool protosizer_all = 63028; + optional bool compare_all = 63029; + optional bool typedecl_all = 63030; + optional bool enumdecl_all = 63031; + + optional bool goproto_registration = 63032; + optional bool messagename_all = 63033; + + optional bool goproto_sizecache_all = 63034; + optional bool goproto_unkeyed_all = 63035; +} + +extend google.protobuf.MessageOptions { + optional bool goproto_getters = 64001; + optional bool goproto_stringer = 64003; + optional bool verbose_equal = 64004; + optional bool face = 64005; + optional bool gostring = 64006; + optional bool populate = 64007; + optional bool stringer = 67008; + optional bool onlyone = 64009; + + optional bool equal = 64013; + optional bool description = 64014; + optional bool testgen = 64015; + optional bool benchgen = 64016; + optional bool marshaler = 64017; + optional bool unmarshaler = 64018; + optional bool stable_marshaler = 64019; + + optional bool sizer = 64020; + + optional bool unsafe_marshaler = 64023; + optional bool unsafe_unmarshaler = 64024; + + optional bool goproto_extensions_map = 64025; + optional bool goproto_unrecognized = 64026; + + optional bool protosizer = 64028; + optional bool compare = 64029; + + optional bool typedecl = 64030; + + optional bool messagename = 64033; + + optional bool goproto_sizecache = 64034; + optional bool goproto_unkeyed = 64035; +} + +extend google.protobuf.FieldOptions { + optional bool nullable = 65001; + optional bool embed = 65002; + optional string customtype = 65003; + optional string customname = 65004; + optional string jsontag = 65005; + optional string moretags = 65006; + optional string casttype = 65007; + optional string castkey = 65008; + optional string castvalue = 65009; + + optional bool stdtime = 65010; + optional bool stdduration = 65011; + optional bool wktpointer = 65012; + + optional string castrepeated = 65013; +} diff --git a/contrib/prometheus-protobufs/LICENSE b/contrib/prometheus-protobufs/LICENSE new file mode 100644 index 00000000000..261eeb9e9f8 --- /dev/null +++ b/contrib/prometheus-protobufs/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/contrib/prometheus-protobufs/README b/contrib/prometheus-protobufs/README new file mode 100644 index 00000000000..c557e59bb93 --- /dev/null +++ b/contrib/prometheus-protobufs/README @@ -0,0 +1,2 @@ +Files "prompb/remote.proto" and "prompb/types.proto" were downloaded from the Prometheus repository: +https://github.com/prometheus/prometheus/tree/main/prompb diff --git a/contrib/prometheus-protobufs/prompb/remote.proto b/contrib/prometheus-protobufs/prompb/remote.proto new file mode 100644 index 00000000000..50bb25e7fac --- /dev/null +++ b/contrib/prometheus-protobufs/prompb/remote.proto @@ -0,0 +1,88 @@ +// Copyright 2016 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; +package prometheus; + +option go_package = "prompb"; + +import "prompb/types.proto"; +import "gogoproto/gogo.proto"; + +message WriteRequest { + repeated prometheus.TimeSeries timeseries = 1 [(gogoproto.nullable) = false]; + // Cortex uses this field to determine the source of the write request. + // We reserve it to avoid any compatibility issues. + reserved 2; + repeated prometheus.MetricMetadata metadata = 3 [(gogoproto.nullable) = false]; +} + +// ReadRequest represents a remote read request. +message ReadRequest { + repeated Query queries = 1; + + enum ResponseType { + // Server will return a single ReadResponse message with matched series that includes list of raw samples. + // It's recommended to use streamed response types instead. + // + // Response headers: + // Content-Type: "application/x-protobuf" + // Content-Encoding: "snappy" + SAMPLES = 0; + // Server will stream a delimited ChunkedReadResponse message that + // contains XOR or HISTOGRAM(!) encoded chunks for a single series. + // Each message is following varint size and fixed size bigendian + // uint32 for CRC32 Castagnoli checksum. + // + // Response headers: + // Content-Type: "application/x-streamed-protobuf; proto=prometheus.ChunkedReadResponse" + // Content-Encoding: "" + STREAMED_XOR_CHUNKS = 1; + } + + // accepted_response_types allows negotiating the content type of the response. + // + // Response types are taken from the list in the FIFO order. If no response type in `accepted_response_types` is + // implemented by server, error is returned. + // For request that do not contain `accepted_response_types` field the SAMPLES response type will be used. + repeated ResponseType accepted_response_types = 2; +} + +// ReadResponse is a response when response_type equals SAMPLES. +message ReadResponse { + // In same order as the request's queries. + repeated QueryResult results = 1; +} + +message Query { + int64 start_timestamp_ms = 1; + int64 end_timestamp_ms = 2; + repeated prometheus.LabelMatcher matchers = 3; + prometheus.ReadHints hints = 4; +} + +message QueryResult { + // Samples within a time series must be ordered by time. + repeated prometheus.TimeSeries timeseries = 1; +} + +// ChunkedReadResponse is a response when response_type equals STREAMED_XOR_CHUNKS. +// We strictly stream full series after series, optionally split by time. This means that a single frame can contain +// partition of the single series, but once a new series is started to be streamed it means that no more chunks will +// be sent for previous one. Series are returned sorted in the same way TSDB block are internally. +message ChunkedReadResponse { + repeated prometheus.ChunkedSeries chunked_series = 1; + + // query_index represents an index of the query from ReadRequest.queries these chunks relates to. + int64 query_index = 2; +} diff --git a/contrib/prometheus-protobufs/prompb/types.proto b/contrib/prometheus-protobufs/prompb/types.proto new file mode 100644 index 00000000000..61fc1e0143e --- /dev/null +++ b/contrib/prometheus-protobufs/prompb/types.proto @@ -0,0 +1,187 @@ +// Copyright 2017 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; +package prometheus; + +option go_package = "prompb"; + +import "gogoproto/gogo.proto"; + +message MetricMetadata { + enum MetricType { + UNKNOWN = 0; + COUNTER = 1; + GAUGE = 2; + HISTOGRAM = 3; + GAUGEHISTOGRAM = 4; + SUMMARY = 5; + INFO = 6; + STATESET = 7; + } + + // Represents the metric type, these match the set from Prometheus. + // Refer to github.com/prometheus/common/model/metadata.go for details. + MetricType type = 1; + string metric_family_name = 2; + string help = 4; + string unit = 5; +} + +message Sample { + double value = 1; + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + int64 timestamp = 2; +} + +message Exemplar { + // Optional, can be empty. + repeated Label labels = 1 [(gogoproto.nullable) = false]; + double value = 2; + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + int64 timestamp = 3; +} + +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +message Histogram { + enum ResetHint { + UNKNOWN = 0; // Need to test for a counter reset explicitly. + YES = 1; // This is the 1st histogram after a counter reset. + NO = 2; // There was no counter reset between this and the previous Histogram. + GAUGE = 3; // This is a gauge histogram where counter resets don't happen. + } + + oneof count { // Count of observations in the histogram. + uint64 count_int = 1; + double count_float = 2; + } + double sum = 3; // Sum of observations in the histogram. + // The schema defines the bucket schema. Currently, valid numbers + // are -4 <= n <= 8. They are all for base-2 bucket schemas, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n logarithmic buckets. Or in other words, each + // bucket boundary is the previous boundary times 2^(2^-n). In the + // future, more bucket schemas may be added using numbers < -4 or > + // 8. + sint32 schema = 4; + double zero_threshold = 5; // Breadth of the zero bucket. + oneof zero_count { // Count in zero bucket. + uint64 zero_count_int = 6; + double zero_count_float = 7; + } + + // Negative Buckets. + repeated BucketSpan negative_spans = 8 [(gogoproto.nullable) = false]; + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 negative_deltas = 9; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double negative_counts = 10; // Absolute count of each bucket. + + // Positive Buckets. + repeated BucketSpan positive_spans = 11 [(gogoproto.nullable) = false]; + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 positive_deltas = 12; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double positive_counts = 13; // Absolute count of each bucket. + + ResetHint reset_hint = 14; + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + int64 timestamp = 15; +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +message BucketSpan { + sint32 offset = 1; // Gap to previous span, or starting point for 1st span (which can be negative). + uint32 length = 2; // Length of consecutive buckets. +} + +// TimeSeries represents samples and labels for a single time series. +message TimeSeries { + // For a timeseries to be valid, and for the samples and exemplars + // to be ingested by the remote system properly, the labels field is required. + repeated Label labels = 1 [(gogoproto.nullable) = false]; + repeated Sample samples = 2 [(gogoproto.nullable) = false]; + repeated Exemplar exemplars = 3 [(gogoproto.nullable) = false]; + repeated Histogram histograms = 4 [(gogoproto.nullable) = false]; +} + +message Label { + string name = 1; + string value = 2; +} + +message Labels { + repeated Label labels = 1 [(gogoproto.nullable) = false]; +} + +// Matcher specifies a rule, which can match or set of labels or not. +message LabelMatcher { + enum Type { + EQ = 0; + NEQ = 1; + RE = 2; + NRE = 3; + } + Type type = 1; + string name = 2; + string value = 3; +} + +message ReadHints { + int64 step_ms = 1; // Query step size in milliseconds. + string func = 2; // String representation of surrounding function or aggregation. + int64 start_ms = 3; // Start time in milliseconds. + int64 end_ms = 4; // End time in milliseconds. + repeated string grouping = 5; // List of label names used in aggregation. + bool by = 6; // Indicate whether it is without or by. + int64 range_ms = 7; // Range vector selector range in milliseconds. +} + +// Chunk represents a TSDB chunk. +// Time range [min, max] is inclusive. +message Chunk { + int64 min_time_ms = 1; + int64 max_time_ms = 2; + + // We require this to match chunkenc.Encoding. + enum Encoding { + UNKNOWN = 0; + XOR = 1; + HISTOGRAM = 2; + FLOAT_HISTOGRAM = 3; + } + Encoding type = 3; + bytes data = 4; +} + +// ChunkedSeries represents single, encoded time series. +message ChunkedSeries { + // Labels should be sorted. + repeated Label labels = 1 [(gogoproto.nullable) = false]; + // Chunks will be in start time order and may overlap. + repeated Chunk chunks = 2 [(gogoproto.nullable) = false]; +} diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b18207e55ad..d985595154c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -607,6 +607,10 @@ if (TARGET ch_contrib::usearch) dbms_target_link_libraries(PUBLIC ch_contrib::usearch) endif() +if (TARGET ch_contrib::prometheus_protobufs) + dbms_target_link_libraries (PUBLIC ch_contrib::prometheus_protobufs) +endif() + if (TARGET ch_rust::skim) dbms_target_include_directories(PRIVATE $) dbms_target_link_libraries(PUBLIC ch_rust::skim) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index ad2ca2652d1..f68701d5d10 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -63,6 +63,7 @@ #cmakedefine01 USE_BCRYPT #cmakedefine01 USE_LIBARCHIVE #cmakedefine01 USE_POCKETFFT +#cmakedefine01 USE_PROMETHEUS_PROTOBUFS /// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. /// That's why we use absolute paths. diff --git a/src/configure_config.cmake b/src/configure_config.cmake index a3f6dae4b87..75f61baa854 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -170,5 +170,8 @@ endif() if (TARGET ch_contrib::pocketfft) set(USE_POCKETFFT 1) endif() +if (TARGET ch_contrib::prometheus_protobufs) + set(USE_PROMETHEUS_PROTOBUFS 1) +endif() set(SOURCE_DIR ${PROJECT_SOURCE_DIR}) From 1ef5bca59164bd4ec00743e0f6a5d7cf17c077ef Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 18 Feb 2024 12:46:21 +0100 Subject: [PATCH 104/107] Fix cmake function PROTOBUF_GENERATE_CPP(): now it returns correct paths in SRCS and HDRS even if input ".proto" files are located in sibling directories. --- .../protobuf_generate.cmake | 51 +++++++++++++++---- 1 file changed, 41 insertions(+), 10 deletions(-) diff --git a/contrib/google-protobuf-cmake/protobuf_generate.cmake b/contrib/google-protobuf-cmake/protobuf_generate.cmake index 3e30b4e40fd..0731a81aeb8 100644 --- a/contrib/google-protobuf-cmake/protobuf_generate.cmake +++ b/contrib/google-protobuf-cmake/protobuf_generate.cmake @@ -157,15 +157,13 @@ function(protobuf_generate) set(_generated_srcs_all) foreach(_proto ${protobuf_generate_PROTOS}) - get_filename_component(_abs_file ${_proto} ABSOLUTE) - get_filename_component(_abs_dir ${_abs_file} DIRECTORY) - get_filename_component(_basename ${_proto} NAME_WE) - file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) - - set(_possible_rel_dir) - if (NOT protobuf_generate_APPEND_PATH) - set(_possible_rel_dir ${_rel_dir}/) - endif() + # The protobuf compiler doesn't return paths to the files it generates so we have to calculate those paths here: + # _abs_file - absolute path to a .proto file, + # _possible_rel_dir - relative path to the .proto file from some import directory specified in Protobuf_IMPORT_DIRS, + # _basename - filename of the .proto file (without path and without extenstion). + get_proto_absolute_path(_abs_file "${_proto}" ${_protobuf_include_path}) + get_proto_relative_path(_possible_rel_dir "${_abs_file}" ${_protobuf_include_path}) + get_filename_component(_basename "${_abs_file}" NAME_WE) set(_generated_srcs) foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) @@ -173,7 +171,7 @@ function(protobuf_generate) endforeach() if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) - set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_descriptor_file "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}.desc") set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") list(APPEND _generated_srcs ${_descriptor_file}) endif() @@ -196,3 +194,36 @@ function(protobuf_generate) target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) endif() endfunction() + +# Calculates the absolute path to a .proto file. +function(get_proto_absolute_path result proto) + cmake_path(IS_ABSOLUTE proto _is_abs_path) + if(_is_abs_path) + set(${result} "${proto}" PARENT_SCOPE) + return() + endif() + foreach(_include_dir ${ARGN}) + if(EXISTS "${_include_dir}/${proto}") + set(${result} "${_include_dir}/${proto}" PARENT_SCOPE) + return() + endif() + endforeach() + message(SEND_ERROR "Not found protobuf ${proto} in Protobuf_IMPORT_DIRS: ${ARGN}") +endfunction() + +# Calculates a relative path to a .proto file. The returned path is relative to one of include directories. +function(get_proto_relative_path result abs_path) + set(${result} "" PARENT_SCOPE) + get_filename_component(_abs_dir "${abs_path}" DIRECTORY) + foreach(_include_dir ${ARGN}) + cmake_path(IS_PREFIX _include_dir "${_abs_dir}" _is_prefix) + if(_is_prefix) + file(RELATIVE_PATH _rel_dir "${_include_dir}" "${_abs_dir}") + if(NOT _rel_dir STREQUAL "") + set(${result} "${_rel_dir}/" PARENT_SCOPE) + endif() + return() + endif() + endforeach() + message(WARNING "Not found protobuf ${abs_path} in Protobuf_IMPORT_DIRS: ${ARGN}") +endfunction() From d777a7a9415c9f630e8df01cd42fc0d424cd4daf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Jul 2024 08:38:09 +0100 Subject: [PATCH 105/107] Reduce one more time --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 6c62ab6def8..d473810bcb8 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -153,7 +153,7 @@ namespace DB M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ - M(Double, gwp_asan_force_sample_probability, 0.0005, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ + M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp From 7180ae03467b05fb0495d744c066d4df758c37a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 5 Jul 2024 13:18:48 +0000 Subject: [PATCH 106/107] Add `use_same_s3_credentials_for_base_backup` to docs --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 2ba50b39934..7c102c38fa6 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,6 +84,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - [`compression_method`](/docs/en/sql-reference/statements/create/table.md/#column-compression-codecs) and compression_level - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` + - `use_same_s3_credentials_for_base_backup`: whether base backup to S3 should inherit credentials from the query. Only works with `S3`. - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` From d3f23c2753ff2b1ac935268c1af0609616381782 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 5 Jul 2024 13:29:34 +0000 Subject: [PATCH 107/107] Bump s2geometry again --- contrib/s2geometry | 2 +- contrib/s2geometry-cmake/CMakeLists.txt | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/s2geometry b/contrib/s2geometry index 0146e2d1355..6522a40338d 160000 --- a/contrib/s2geometry +++ b/contrib/s2geometry @@ -1 +1 @@ -Subproject commit 0146e2d1355828f8f633cb050948250ad7406c57 +Subproject commit 6522a40338d58752c2a4227a3fc2bc4107c73e43 diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 5eabe71b538..48562b8cead 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -1,7 +1,6 @@ option(ENABLE_S2_GEOMETRY "Enable S2 Geometry" ${ENABLE_LIBRARIES}) -# ARCH_S390X broke upstream, it can be re-enabled once https://github.com/google/s2geometry/pull/372 is merged -if (NOT ENABLE_S2_GEOMETRY OR ARCH_S390X) +if (NOT ENABLE_S2_GEOMETRY) message(STATUS "Not using S2 Geometry") return() endif()